You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/06/30 17:42:10 UTC

[23/50] [abbrv] incubator-carbondata git commit: [Issue-643] Column Property addition, extract interface for dictionary (#641)

[Issue-643] Column Property addition, extract interface for dictionary (#641)

1.Added interface to get dictionary service
2.Modified schema.thrift to add column properties
3. Describe formatted shows column properties

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

Branch: refs/heads/master
Commit: 38d84e0e2d89744290d50d46750bbadf5ba9c693
Parents: 5f86244
Author: ashokblend <as...@gmail.com>
Authored: Mon Jun 27 16:41:52 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 27 16:41:52 2016 +0530

----------------------------------------------------------------------
 .../common/ext/ColumnUniqueIdGenerator.java     |  41 +++++++
 .../common/ext/DictionaryFactory.java           | 119 +++++++++++++++++++
 .../org/carbondata/common/ext/PathFactory.java  |  48 ++++++++
 .../common/factory/CarbonCommonFactory.java     |  54 +++++++++
 .../dictionary/AbstractDictionaryCache.java     |  43 ++++---
 .../cache/dictionary/DictionaryCacheLoader.java |   7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  28 ++---
 .../DictionaryColumnUniqueIdentifier.java       |   9 +-
 .../dictionary/ForwardDictionaryCache.java      |  11 +-
 .../dictionary/ReverseDictionaryCache.java      |  11 +-
 .../core/carbon/ColumnIdentifier.java           |  91 ++++++++++++++
 .../ThriftWrapperSchemaConverterImpl.java       |   4 +
 .../schema/table/column/CarbonColumn.java       |  23 ++++
 .../schema/table/column/ColumnSchema.java       |  49 ++++++++
 .../core/carbon/path/CarbonTablePath.java       |  20 ++--
 .../core/constants/CarbonCommonConstants.java   |   8 ++
 .../CarbonDictionaryMetadataReaderImpl.java     |   9 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |  20 +++-
 .../CarbonDictionarySortIndexReaderImpl.java    |   7 +-
 .../core/service/ColumnUniqueIdService.java     |  34 ++++++
 .../core/service/DictionaryService.java         |  92 ++++++++++++++
 .../carbondata/core/service/PathService.java    |  38 ++++++
 .../org/carbondata/core/util/CarbonUtil.java    |  64 ++++++++++
 .../core/writer/CarbonDictionaryWriterImpl.java |  41 +++++--
 .../CarbonDictionarySortIndexWriterImpl.java    |  19 +--
 .../query/carbon/executor/util/QueryUtil.java   |  10 +-
 .../filters/measurefilter/util/FilterUtil.java  |   2 +-
 .../dictionary/AbstractDictionaryCacheTest.java |   7 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |   4 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   4 +-
 ...CarbonDictionarySortIndexReaderImplTest.java |   6 +-
 .../writer/CarbonDictionaryWriterImplTest.java  |   9 +-
 ...CarbonDictionarySortIndexWriterImplTest.java |  13 +-
 .../examples/GenerateDictionaryExample.scala    |   4 +-
 format/src/main/thrift/schema.thrift            |  13 +-
 .../AbstractDictionaryDecodedReadSupport.java   |   4 +-
 .../hadoop/test/util/StoreCreator.java          |   8 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |   3 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |   7 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  | 107 +++++++++++++----
 .../execution/command/carbonTableSchema.scala   |  74 ++++++++----
 .../spark/sql/hive/CarbonStrategies.scala       |  10 +-
 .../apache/spark/sql/hive/CarbonStrategy.scala  |   4 +-
 .../spark/CarbonColumnValidator.scala           |  36 ++++++
 .../carbondata/spark/CarbonSparkFactory.scala   |  63 ++++++++++
 .../spark/DictionaryDetailHelper.scala          |  66 ++++++++++
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   7 +-
 .../org/carbondata/spark/util/CommonUtil.scala  |  79 +++++++++++-
 .../spark/util/GlobalDictionaryUtil.scala       |  85 +++++--------
 .../spark/load/CarbonLoaderUtilTest.java        |   4 +-
 .../validation/FileFooterValidator.java         |   6 +-
 .../apache/spark/sql/TestCarbonSqlParser.scala  |  37 +++---
 ...ataWithMalformedCarbonCommandException.scala |   4 +-
 .../ColumnPropertyValidationTestCase.scala      |  28 +++++
 .../util/GlobalDictionaryUtilTestCase.scala     |   7 +-
 .../graphgenerator/GraphGenerator.java          |   3 +
 .../configuration/GraphConfigurationInfo.java   |  10 ++
 .../processing/schema/metadata/ColumnsInfo.java |  10 ++
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      |  36 ++++++
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |   1 +
 .../FileStoreSurrogateKeyGenForCSV.java         |  12 +-
 .../processing/util/CarbonSchemaParser.java     |  44 +++++++
 62 files changed, 1455 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java b/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
new file mode 100644
index 0000000..0ef3950
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
@@ -0,0 +1,41 @@
+/*
+ * 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.carbondata.common.ext;
+
+import java.util.UUID;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.carbondata.core.service.ColumnUniqueIdService;
+
+/**
+ * It returns unique id given column
+ */
+public class ColumnUniqueIdGenerator implements ColumnUniqueIdService {
+
+  private static ColumnUniqueIdService columnUniqueIdService = new ColumnUniqueIdGenerator();
+
+  @Override public String generateUniqueId(String schemaName, ColumnSchema columnSchema) {
+    return UUID.randomUUID().toString();
+  }
+
+  public static ColumnUniqueIdService getInstance() {
+    return columnUniqueIdService;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java b/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java
new file mode 100644
index 0000000..944f772
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java
@@ -0,0 +1,119 @@
+/*
+ * 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.carbondata.common.ext;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
+import org.carbondata.core.reader.CarbonDictionaryReader;
+import org.carbondata.core.reader.CarbonDictionaryReaderImpl;
+import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
+import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
+import org.carbondata.core.service.DictionaryService;
+import org.carbondata.core.writer.CarbonDictionaryWriter;
+import org.carbondata.core.writer.CarbonDictionaryWriterImpl;
+import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
+import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
+
+/**
+ * service to get dictionary reader and writer
+ */
+public class DictionaryFactory implements DictionaryService {
+
+  private static DictionaryService dictService = new DictionaryFactory();
+
+  /**
+   * get dictionary writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionaryWriter getDictionaryWriter(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionaryWriterImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
+  }
+
+  /**
+   * get dictionary sort index writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, columnIdentifier,
+        carbonStorePath);
+  }
+
+  /**
+   * get dictionary metadata reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionaryMetadataReader getDictionaryMetadataReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionaryMetadataReaderImpl(carbonStorePath, carbonTableIdentifier,
+        columnIdentifier);
+  }
+
+  /**
+   * get dictionary reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionaryReader getDictionaryReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionaryReaderImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
+  }
+
+  /**
+   * get dictionary sort index reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionarySortIndexReader getDictionarySortIndexReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier,
+        carbonStorePath);
+  }
+
+  public static DictionaryService getInstance() {
+    return dictService;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/ext/PathFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/PathFactory.java b/core/src/main/java/org/carbondata/common/ext/PathFactory.java
new file mode 100644
index 0000000..8b64aec
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/ext/PathFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.carbondata.common.ext;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.carbon.path.CarbonStorePath;
+import org.carbondata.core.carbon.path.CarbonTablePath;
+import org.carbondata.core.service.PathService;
+
+/**
+ * Create helper to get path details
+ */
+public class PathFactory implements PathService {
+
+  private static PathService pathService = new PathFactory();
+
+  /**
+   * @param columnIdentifier
+   * @param storeLocation
+   * @param tableIdentifier
+   * @return store path related to tables
+   */
+  @Override public CarbonTablePath getCarbonTablePath(ColumnIdentifier columnIdentifier,
+      String storeLocation, CarbonTableIdentifier tableIdentifier) {
+    return CarbonStorePath.getCarbonTablePath(storeLocation, tableIdentifier);
+  }
+
+  public static PathService getInstance() {
+    return pathService;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java b/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java
new file mode 100644
index 0000000..dfa14f9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java
@@ -0,0 +1,54 @@
+/*
+ * 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.carbondata.common.factory;
+
+import org.carbondata.common.ext.ColumnUniqueIdGenerator;
+import org.carbondata.common.ext.DictionaryFactory;
+import org.carbondata.common.ext.PathFactory;
+import org.carbondata.core.service.ColumnUniqueIdService;
+import org.carbondata.core.service.DictionaryService;
+import org.carbondata.core.service.PathService;
+
+/**
+ * Interface to get services
+ */
+public class CarbonCommonFactory {
+
+  /**
+   * @return dictionary service
+   */
+  public static DictionaryService getDictionaryService() {
+    return DictionaryFactory.getInstance();
+  }
+
+  /**
+   * @return path service
+   */
+  public static PathService getPathService() {
+    return PathFactory.getInstance();
+  }
+
+  /**
+   * @return unique id generator
+   */
+  public static ColumnUniqueIdService getColumnUniqueIdGenerator() {
+    return ColumnUniqueIdGenerator.getInstance();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index 690f125..390bcc8 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -22,16 +22,18 @@ package org.carbondata.core.cache.dictionary;
 import java.io.IOException;
 import java.util.List;
 
+import org.carbondata.common.factory.CarbonCommonFactory;
 import org.carbondata.core.cache.Cache;
 import org.carbondata.core.cache.CacheType;
 import org.carbondata.core.cache.CarbonLRUCache;
-import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
-import org.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
+import org.carbondata.core.service.DictionaryService;
+import org.carbondata.core.service.PathService;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
@@ -91,12 +93,17 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   protected boolean isFileExistsForGivenColumn(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath,
-        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
-    String dictionaryFilePath = carbonTablePath
-        .getDictionaryFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier());
-    String dictionaryMetadataFilePath = carbonTablePath
-        .getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier());
+    PathService pathService = CarbonCommonFactory.getPathService();
+    CarbonTablePath carbonTablePath = pathService
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier(), carbonStorePath,
+            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
+
+    String dictionaryFilePath =
+        carbonTablePath.getDictionaryFilePath(dictionaryColumnUniqueIdentifier
+            .getColumnIdentifier().getColumnId());
+    String dictionaryMetadataFilePath =
+        carbonTablePath.getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier
+            .getColumnIdentifier().getColumnId());
     // check if both dictionary and its metadata file exists for a given column
     return CarbonUtil.isFileExists(dictionaryFilePath) && CarbonUtil
         .isFileExists(dictionaryMetadataFilePath);
@@ -111,10 +118,11 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   protected CarbonDictionaryColumnMetaChunk readLastChunkFromDictionaryMetadataFile(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.carbonStorePath,
-            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier.getColumnIdentifier());
+    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
+    CarbonDictionaryMetadataReader columnMetadataReaderImpl = dictService
+        .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
+            dictionaryColumnUniqueIdentifier.getColumnIdentifier(), carbonStorePath);
+
     CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
     // read metadata file
     try {
@@ -148,10 +156,13 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   private CarbonFile getDictionaryMetaCarbonFile(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath,
-        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
-    String dictionaryFilePath = carbonTablePath
-        .getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier());
+    PathService pathService = CarbonCommonFactory.getPathService();
+    CarbonTablePath carbonTablePath = pathService
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier(), carbonStorePath,
+            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
+    String dictionaryFilePath =
+        carbonTablePath.getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier
+            .getColumnIdentifier().getColumnId());
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryFilePath);
     CarbonFile carbonFile = FileFactory.getCarbonFile(dictionaryFilePath, fileType);
     return carbonFile;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
index 54431a7..a28d58d 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
@@ -21,6 +21,8 @@ package org.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
 
+import org.carbondata.core.carbon.ColumnIdentifier;
+
 public interface DictionaryCacheLoader {
 
   /**
@@ -37,6 +39,7 @@ public interface DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  void load(DictionaryInfo dictionaryInfo, String columnIdentifier, long dictionaryChunkStartOffset,
-      long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException;
+  void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
+      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
+      throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index 887d8b3..770cb07 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -22,11 +22,12 @@ package org.carbondata.core.cache.dictionary;
 import java.io.IOException;
 import java.util.List;
 
+import org.carbondata.common.factory.CarbonCommonFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.reader.CarbonDictionaryReader;
-import org.carbondata.core.reader.CarbonDictionaryReaderImpl;
 import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
-import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
+import org.carbondata.core.service.DictionaryService;
 
 /**
  * This class is responsible for loading the dictionary data for given columns
@@ -67,7 +68,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  @Override public void load(DictionaryInfo dictionaryInfo, String columnIdentifier,
+  @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
       long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException {
     List<byte[]> dictionaryChunk =
@@ -87,7 +88,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @return list of dictionary value
    * @throws IOException
    */
-  private List<byte[]> load(String columnIdentifier, long startOffset, long endOffset)
+  private List<byte[]> load(ColumnIdentifier columnIdentifier, long startOffset, long endOffset)
       throws IOException {
     CarbonDictionaryReader dictionaryReader = getDictionaryReader(columnIdentifier);
     List<byte[]> dictionaryValue = null;
@@ -106,7 +107,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @param columnIdentifier
    * @throws IOException
    */
-  private void readSortIndexFile(DictionaryInfo dictionaryInfo, String columnIdentifier)
+  private void readSortIndexFile(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier)
       throws IOException {
     CarbonDictionarySortIndexReader sortIndexReader = getSortIndexReader(columnIdentifier);
     try {
@@ -123,20 +124,19 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @param columnIdentifier unique column identifier
    * @return carbon dictionary reader instance
    */
-  private CarbonDictionaryReader getDictionaryReader(String columnIdentifier) {
-    CarbonDictionaryReader dictionaryReader =
-        new CarbonDictionaryReaderImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
-    return dictionaryReader;
+  private CarbonDictionaryReader getDictionaryReader(ColumnIdentifier columnIdentifier) {
+    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
+    return dictService
+        .getDictionaryReader(carbonTableIdentifier, columnIdentifier, carbonStorePath);
   }
 
   /**
    * @param columnIdentifier unique column identifier
    * @return sort index reader instance
    */
-  private CarbonDictionarySortIndexReader getSortIndexReader(String columnIdentifier) {
-    CarbonDictionarySortIndexReader sortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier,
-            carbonStorePath);
-    return sortIndexReader;
+  private CarbonDictionarySortIndexReader getSortIndexReader(ColumnIdentifier columnIdentifier) {
+    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
+    return dictService
+        .getDictionarySortIndexReader(carbonTableIdentifier, columnIdentifier, carbonStorePath);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
index ff3804a..dea789f 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
@@ -20,6 +20,7 @@
 package org.carbondata.core.cache.dictionary;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 
 /**
@@ -35,7 +36,7 @@ public class DictionaryColumnUniqueIdentifier {
   /**
    * unique column id
    */
-  private String columnIdentifier;
+  private ColumnIdentifier columnIdentifier;
 
   private DataType dataType;
 
@@ -47,7 +48,7 @@ public class DictionaryColumnUniqueIdentifier {
    * @param columnIdentifier
    */
   public DictionaryColumnUniqueIdentifier(CarbonTableIdentifier carbonTableIdentifier,
-      String columnIdentifier) {
+      ColumnIdentifier columnIdentifier) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
   }
@@ -61,7 +62,7 @@ public class DictionaryColumnUniqueIdentifier {
    * @param dataType
    */
   public DictionaryColumnUniqueIdentifier(CarbonTableIdentifier carbonTableIdentifier,
-      String columnIdentifier, DataType dataType) {
+      ColumnIdentifier columnIdentifier, DataType dataType) {
     this(carbonTableIdentifier, columnIdentifier);
     this.dataType = dataType;
   }
@@ -80,7 +81,7 @@ public class DictionaryColumnUniqueIdentifier {
   /**
    * @return columnIdentifier
    */
-  public String getColumnIdentifier() {
+  public ColumnIdentifier getColumnIdentifier() {
     return columnIdentifier;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java b/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
index d90efd8..bee7714 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
@@ -131,7 +131,7 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     Dictionary forwardDictionary = null;
     ColumnDictionaryInfo columnDictionaryInfo = (ColumnDictionaryInfo) carbonLRUCache.get(
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
             CacheType.FORWARD_DICTIONARY));
     if (null != columnDictionaryInfo) {
       forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
@@ -148,8 +148,9 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
    */
   @Override public void invalidate(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    carbonLRUCache.remove(getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
-        CacheType.FORWARD_DICTIONARY));
+    carbonLRUCache.remove(
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
+            CacheType.FORWARD_DICTIONARY));
   }
 
   /**
@@ -172,12 +173,12 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
           "Either dictionary or its metadata does not exist for column identifier :: "
               + dictionaryColumnUniqueIdentifier.getColumnIdentifier());
     }
-    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier();
+    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId();
     ColumnDictionaryInfo columnDictionaryInfo =
         getColumnDictionaryInfo(dictionaryColumnUniqueIdentifier, columnIdentifier);
     // load sort index file in case of forward dictionary
     checkAndLoadDictionaryData(dictionaryColumnUniqueIdentifier, columnDictionaryInfo,
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
             CacheType.FORWARD_DICTIONARY), true);
     forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
     return forwardDictionary;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java b/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
index 9b3db5d..6e49183 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
@@ -132,7 +132,7 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
     Dictionary reverseDictionary = null;
     ColumnReverseDictionaryInfo columnReverseDictionaryInfo =
         (ColumnReverseDictionaryInfo) carbonLRUCache.get(
-            getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+            getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
                 CacheType.REVERSE_DICTIONARY));
     if (null != columnReverseDictionaryInfo) {
       reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
@@ -149,8 +149,9 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
    */
   @Override public void invalidate(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    carbonLRUCache.remove(getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
-        CacheType.REVERSE_DICTIONARY));
+    carbonLRUCache.remove(
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
+            CacheType.REVERSE_DICTIONARY));
   }
 
   /**
@@ -173,12 +174,12 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
           "Either dictionary or its metadata does not exist for column identifier :: "
               + dictionaryColumnUniqueIdentifier.getColumnIdentifier());
     }
-    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier();
+    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId();
     ColumnReverseDictionaryInfo columnReverseDictionaryInfo =
         getColumnReverseDictionaryInfo(dictionaryColumnUniqueIdentifier, columnIdentifier);
     // do not load sort index file for reverse dictionary
     checkAndLoadDictionaryData(dictionaryColumnUniqueIdentifier, columnReverseDictionaryInfo,
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
             CacheType.REVERSE_DICTIONARY), false);
     reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
     return reverseDictionary;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java b/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
new file mode 100644
index 0000000..928f53b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
@@ -0,0 +1,91 @@
+package org.carbondata.core.carbon;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+
+/**
+ * Column unique identifier
+ */
+public class ColumnIdentifier implements Serializable {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * column id
+   */
+  private String columnId;
+
+  /**
+   * column properties
+   */
+  private Map<String, String> columnProperties;
+
+  private DataType dataType;
+
+  /**
+   * @param columnId
+   * @param columnProperties
+   */
+  public ColumnIdentifier(String columnId, Map<String, String> columnProperties,
+      DataType dataType) {
+    this.columnId = columnId;
+    this.columnProperties = columnProperties;
+    this.dataType = dataType;
+  }
+
+  /**
+   * @return columnId
+   */
+  public String getColumnId() {
+    return columnId;
+  }
+
+  /**
+   * @param columnProperty
+   * @return
+   */
+  public String getColumnProperty(String columnProperty) {
+    if (null != columnProperties) {
+      return columnProperties.get(columnProperty);
+    }
+    return null;
+  }
+
+  public DataType getDataType() {
+    return this.dataType;
+  }
+
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((columnId == null) ? 0 : columnId.hashCode());
+    return result;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    ColumnIdentifier other = (ColumnIdentifier) obj;
+    if (columnId == null) {
+      if (other.columnId != null) {
+        return false;
+      }
+    } else if (!columnId.equals(other.columnId)) {
+      return false;
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 249d2a5..23394ba 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -159,7 +159,9 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     thriftColumnSchema.setPrecision(wrapperColumnSchema.getPrecision());
     thriftColumnSchema.setNum_child(wrapperColumnSchema.getNumberOfChild());
     thriftColumnSchema.setDefault_value(wrapperColumnSchema.getDefaultValue());
+    thriftColumnSchema.setColumnProperties(wrapperColumnSchema.getColumnProperties());
     thriftColumnSchema.setInvisible(wrapperColumnSchema.isInvisible());
+    thriftColumnSchema.setColumnReferenceId(wrapperColumnSchema.getColumnReferenceId());
     return thriftColumnSchema;
   }
 
@@ -324,7 +326,9 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     wrapperColumnSchema.setScale(externalColumnSchema.getScale());
     wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
     wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
+    wrapperColumnSchema.setColumnProperties(externalColumnSchema.getColumnProperties());
     wrapperColumnSchema.setInvisible(externalColumnSchema.isInvisible());
+    wrapperColumnSchema.setColumnReferenceId(externalColumnSchema.getColumnReferenceId());
     return wrapperColumnSchema;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
index 331c661..04008ab 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
@@ -21,7 +21,9 @@ package org.carbondata.core.carbon.metadata.schema.table.column;
 
 import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
 
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
 
@@ -48,9 +50,16 @@ public class CarbonColumn implements Serializable {
    */
   protected byte[] defaultValue;
 
+  /**
+   * Column identifier
+   */
+  protected ColumnIdentifier columnIdentifier;
+
   public CarbonColumn(ColumnSchema columnSchema, int ordinal) {
     this.columnSchema = columnSchema;
     this.ordinal = ordinal;
+    this.columnIdentifier =
+        new ColumnIdentifier(getColumnId(), getColumnProperties(), getDataType());
   }
 
   /**
@@ -142,4 +151,18 @@ public class CarbonColumn implements Serializable {
   public ColumnSchema getColumnSchema() {
     return this.columnSchema;
   }
+
+  /**
+   * @return columnproperty
+   */
+  public Map<String, String> getColumnProperties() {
+    return this.columnSchema.getColumnProperties();
+  }
+
+  /**
+   * @return columnIdentifier
+   */
+  public ColumnIdentifier getColumnIdentifier() {
+    return this.columnIdentifier;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
index f48c53f..5c54ad3 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
@@ -20,6 +20,7 @@ package org.carbondata.core.carbon.metadata.schema.table.column;
 
 import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
 
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
@@ -54,6 +55,11 @@ public class ColumnSchema implements Serializable {
   private String columnUniqueId;
 
   /**
+   * column reference id
+   */
+  private String columnReferenceId;
+
+  /**
    * whether it is stored as columnar format or row format
    */
   private boolean isColumnar = true;
@@ -100,6 +106,11 @@ public class ColumnSchema implements Serializable {
   private byte[] defaultValue;
 
   /**
+   * Column properties
+   */
+  private Map<String, String> columnProperties;
+
+  /**
    * used to define the column visibility of column default is false
    */
   private boolean invisible = false;
@@ -332,6 +343,30 @@ public class ColumnSchema implements Serializable {
   }
 
   /**
+   * @param columnProperties
+   */
+  public void setColumnProperties(Map<String, String> columnProperties) {
+    this.columnProperties = columnProperties;
+  }
+
+  /**
+   * @param property
+   * @return
+   */
+  public String getColumnProperty(String property) {
+    if (null != columnProperties) {
+      return columnProperties.get(property);
+    }
+    return null;
+  }
+
+  /**
+   * return columnproperties
+   */
+  public Map<String, String> getColumnProperties() {
+    return columnProperties;
+  }
+  /**
    * return the visibility
    * @return
    */
@@ -347,4 +382,18 @@ public class ColumnSchema implements Serializable {
     this.invisible = invisible;
   }
 
+  /**
+   * @return columnReferenceId
+   */
+  public String getColumnReferenceId() {
+    return columnReferenceId;
+  }
+
+  /**
+   * @param columnReferenceId
+   */
+  public void setColumnReferenceId(String columnReferenceId) {
+    this.columnReferenceId = columnReferenceId;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
index 5f862bc..538a16b 100644
--- a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
+++ b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
@@ -35,21 +35,21 @@ import org.apache.hadoop.fs.Path;
  */
 public class CarbonTablePath extends Path {
 
-  private static final String METADATA_DIR = "Metadata";
+  protected static final String METADATA_DIR = "Metadata";
   private static final String DICTIONARY_EXT = ".dict";
   private static final String DICTIONARY_META_EXT = ".dictmeta";
   private static final String SORT_INDEX_EXT = ".sortindex";
-  private static final String SCHEMA_FILE = "schema";
-  private static final String TABLE_STATUS_FILE = "tablestatus";
-  private static final String FACT_DIR = "Fact";
-  private static final String AGGREGATE_TABLE_PREFIX = "Agg";
-  private static final String SEGMENT_PREFIX = "Segment_";
-  private static final String PARTITION_PREFIX = "Part";
-  private static final String CARBON_DATA_EXT = ".carbondata";
-  private static final String DATA_PART_PREFIX = "part";
+  protected static final String SCHEMA_FILE = "schema";
+  protected static final String TABLE_STATUS_FILE = "tablestatus";
+  protected static final String FACT_DIR = "Fact";
+  protected static final String AGGREGATE_TABLE_PREFIX = "Agg";
+  protected static final String SEGMENT_PREFIX = "Segment_";
+  protected static final String PARTITION_PREFIX = "Part";
+  protected static final String CARBON_DATA_EXT = ".carbondata";
+  protected static final String DATA_PART_PREFIX = "part";
   private static final String INDEX_FILE_EXT = ".carbonindex";
 
-  private String tablePath;
+  protected String tablePath;
 
   public CarbonTablePath(String tablePathString) {
     super(tablePathString);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
index 473e764..39b80d1 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -719,6 +719,14 @@ public final class CarbonCommonConstants {
   public static final String CARBON_MERGE_SORT_PREFETCH = "carbon.merge.sort.prefetch";
   public static final String CARBON_MERGE_SORT_PREFETCH_DEFAULT = "true";
 
+  // tblproperties
+  public static final String COLUMN_GROUPS = "column_groups";
+  public static final String DICTIONARY_EXCLUDE = "dictionary_exclude";
+  public static final String DICTIONARY_INCLUDE = "dictionary_include";
+  public static final String PARTITIONCLASS = "partitionclass";
+  public static final String PARTITIONCOUNT = "partitioncount";
+  public static final String COLUMN_PROPERTIES = "columnproperties";
+
   /**
    * this variable is to enable/disable identify high cardinality during first data loading
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
index f76cd49..3b4d271 100644
--- a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -47,9 +48,9 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   private String hdfsStorePath;
 
   /**
-   * column name
+   * column identifier
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * dictionary metadata file path
@@ -69,7 +70,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionaryMetadataReaderImpl(String hdfsStorePath,
-      CarbonTableIdentifier carbonTableIdentifier, String columnIdentifier) {
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier) {
     this.hdfsStorePath = hdfsStorePath;
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
@@ -155,7 +156,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
     this.columnDictionaryMetadataFilePath =
-        carbonTablePath.getDictionaryMetaFilePath(columnIdentifier);
+        carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index d773f68..addedf6 100644
--- a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -51,7 +52,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   /**
    * column name
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * dictionary file path
@@ -71,7 +72,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionaryReaderImpl(String hdfsStorePath,
-      CarbonTableIdentifier carbonTableIdentifier, String columnIdentifier) {
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier) {
     this.hdfsStorePath = hdfsStorePath;
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
@@ -199,7 +200,8 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   private void initFileLocation() {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
-    this.columnDictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier);
+    this.columnDictionaryFilePath = carbonTablePath
+        .getDictionaryFilePath(columnIdentifier.getColumnId());
   }
 
   /**
@@ -266,9 +268,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @throws IOException read and close method throws IO exception
    */
   private List<CarbonDictionaryColumnMetaChunk> readDictionaryMetadataFile() throws IOException {
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.hdfsStorePath, this.carbonTableIdentifier,
-            this.columnIdentifier);
+    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
     List<CarbonDictionaryColumnMetaChunk> dictionaryMetaChunkList = null;
     // read metadata file
     try {
@@ -281,6 +281,14 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   }
 
   /**
+   * @return
+   */
+  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
+    return new CarbonDictionaryMetadataReaderImpl(this.hdfsStorePath, carbonTableIdentifier,
+        this.columnIdentifier);
+  }
+
+  /**
    * This method will open the dictionary file stream for reading
    *
    * @throws IOException thrift reader open method throws IOException

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
index b84ae54..eb3f8bd 100644
--- a/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.reader.ThriftReader;
@@ -44,7 +45,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   /**
    * column name
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * hdfs store location
@@ -78,7 +79,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
    * @param carbonStorePath       carbon store path
    */
   public CarbonDictionarySortIndexReaderImpl(final CarbonTableIdentifier carbonTableIdentifier,
-      final String columnIdentifier, final String carbonStorePath) {
+      final ColumnIdentifier columnIdentifier, final String carbonStorePath) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.carbonStorePath = carbonStorePath;
@@ -144,7 +145,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   private void init() throws IOException {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
-    this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier);
+    this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier.getColumnId());
     openThriftReader();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java b/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
new file mode 100644
index 0000000..9c65e3f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
@@ -0,0 +1,34 @@
+/*
+ * 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.carbondata.core.service;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Column Unique id generator
+ */
+public interface ColumnUniqueIdService {
+
+  /**
+   * @param schemaName
+   * @param columnSchema
+   * @return generate unique id
+   */
+  public String generateUniqueId(String schemaName, ColumnSchema columnSchema);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/service/DictionaryService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/DictionaryService.java b/core/src/main/java/org/carbondata/core/service/DictionaryService.java
new file mode 100644
index 0000000..97afe29
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/service/DictionaryService.java
@@ -0,0 +1,92 @@
+/*
+ * 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.carbondata.core.service;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
+import org.carbondata.core.reader.CarbonDictionaryReader;
+import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
+import org.carbondata.core.writer.CarbonDictionaryWriter;
+import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
+
+/**
+ * Dictionary service to get writer and reader
+ */
+public interface DictionaryService {
+
+  /**
+   * get dictionary writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionaryWriter getDictionaryWriter(CarbonTableIdentifier carbonTableIdentifier,
+      ColumnIdentifier columnIdentifier, String carbonStorePath);
+
+  /**
+   * get dictionary sort index writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath);
+
+  /**
+   * get dictionary metadata reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionaryMetadataReader getDictionaryMetadataReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath);
+
+  /**
+   * get dictionary reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionaryReader getDictionaryReader(CarbonTableIdentifier carbonTableIdentifier,
+      ColumnIdentifier columnIdentifier, String carbonStorePath);
+
+  /**
+   * get dictionary sort index reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionarySortIndexReader getDictionarySortIndexReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/service/PathService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/PathService.java b/core/src/main/java/org/carbondata/core/service/PathService.java
new file mode 100644
index 0000000..7ef3a45
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/service/PathService.java
@@ -0,0 +1,38 @@
+/*
+ * 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.carbondata.core.service;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.carbon.path.CarbonTablePath;
+
+/**
+ * Create helper to get path details
+ */
+public interface PathService {
+
+  /**
+   * @param columnIdentifier
+   * @param storeLocation
+   * @param tableIdentifier
+   * @return store path related to tables
+   */
+  CarbonTablePath getCarbonTablePath(ColumnIdentifier columnIdentifier, String storeLocation,
+      CarbonTableIdentifier tableIdentifier);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 0f94c5a..886e136 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -1657,6 +1657,70 @@ public final class CarbonUtil {
   }
 
   /**
+   * convert from wrapper to external data type
+   *
+   * @param dataType
+   * @return
+   */
+  public static org.carbondata.format.DataType fromWrapperToExternalDataType(DataType dataType) {
+
+    if (null == dataType) {
+      return null;
+    }
+    switch (dataType) {
+      case STRING:
+        return org.carbondata.format.DataType.STRING;
+      case INT:
+        return org.carbondata.format.DataType.INT;
+      case LONG:
+        return org.carbondata.format.DataType.LONG;
+      case DOUBLE:
+        return org.carbondata.format.DataType.DOUBLE;
+      case DECIMAL:
+        return org.carbondata.format.DataType.DECIMAL;
+      case TIMESTAMP:
+        return org.carbondata.format.DataType.TIMESTAMP;
+      case ARRAY:
+        return org.carbondata.format.DataType.ARRAY;
+      case STRUCT:
+        return org.carbondata.format.DataType.STRUCT;
+      default:
+        return org.carbondata.format.DataType.STRING;
+    }
+  }
+
+  /**
+   * convert from external to wrapper data type
+   *
+   * @param dataType
+   * @return
+   */
+  public static DataType fromExternalToWrapperDataType(org.carbondata.format.DataType dataType) {
+    if (null == dataType) {
+      return null;
+    }
+    switch (dataType) {
+      case STRING:
+        return DataType.STRING;
+      case INT:
+        return DataType.INT;
+      case LONG:
+        return DataType.LONG;
+      case DOUBLE:
+        return DataType.DOUBLE;
+      case DECIMAL:
+        return DataType.DECIMAL;
+      case TIMESTAMP:
+        return DataType.TIMESTAMP;
+      case ARRAY:
+        return DataType.ARRAY;
+      case STRUCT:
+        return DataType.STRUCT;
+      default:
+        return DataType.STRING;
+    }
+  }
+  /**
    * @param dictionaryColumnCardinality
    * @param wrapperColumnSchemaList
    * @return It returns formatted cardinality by adding -1 value for NoDictionary columns

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index c6bc343..22e5bbe 100644
--- a/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -25,16 +25,19 @@ import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.carbondata.common.factory.CarbonCommonFactory;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
-import org.carbondata.core.carbon.path.CarbonStorePath;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
+import org.carbondata.core.service.PathService;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.format.ColumnDictionaryChunk;
@@ -56,7 +59,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   /**
    * carbon type identifier
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * list which will hold values upto maximum of one dictionary chunk size
@@ -74,14 +77,14 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   private ThriftWriter dictionaryThriftWriter;
 
   /**
-   * column name
+   * column identifier
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * HDFS store path
    */
-  private String hdfsStorePath;
+  protected String hdfsStorePath;
 
   /**
    * dictionary file path
@@ -134,7 +137,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionaryWriterImpl(String hdfsStorePath,
-      CarbonTableIdentifier carbonTableIdentifier, String columnIdentifier) {
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.hdfsStorePath = hdfsStorePath;
@@ -236,10 +239,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    */
   private void init() throws IOException {
     initDictionaryChunkSize();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier);
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier);
+    initPaths();
     if (CarbonUtil.isFileExists(this.dictionaryFilePath)) {
       this.chunk_start_offset = CarbonUtil.getFileSize(this.dictionaryFilePath);
       validateDictionaryFileOffsetWithLastSegmentEntryOffset();
@@ -248,6 +248,15 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
     createChunkList();
   }
 
+  protected void initPaths() {
+    PathService pathService = CarbonCommonFactory.getPathService();
+    CarbonTablePath carbonTablePath = pathService.getCarbonTablePath(columnIdentifier,
+            this.hdfsStorePath, carbonTableIdentifier);
+    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId());
+    this.dictionaryMetaFilePath =
+        carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
+  }
+
   /**
    * initialize the value of dictionary chunk that can be kept in memory at a time
    */
@@ -382,9 +391,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   private CarbonDictionaryColumnMetaChunk getChunkMetaObjectForLastSegmentEntry()
       throws IOException {
     CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.hdfsStorePath, this.carbonTableIdentifier,
-            this.columnIdentifier);
+    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
     try {
       // read the last segment entry for dictionary metadata
       carbonDictionaryColumnMetaChunk =
@@ -395,4 +402,12 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
     }
     return carbonDictionaryColumnMetaChunk;
   }
+
+  /**
+   * @return
+   */
+  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
+    return new CarbonDictionaryMetadataReaderImpl(hdfsStorePath, carbonTableIdentifier,
+        columnIdentifier);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java b/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
index 5ae87cd..d28b176 100644
--- a/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
+++ b/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.util.CarbonUtil;
@@ -39,17 +40,17 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
   /**
    * carbonTable Identifier holding the info of databaseName and tableName
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * column name
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * carbon store location
    */
-  private String carbonStorePath;
+  protected String carbonStorePath;
   /**
    * Path of dictionary sort index file for which the sortIndex to be written
    */
@@ -76,7 +77,7 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionarySortIndexWriterImpl(final CarbonTableIdentifier carbonTableIdentifier,
-      final String columnIdentifier, final String carbonStorePath) {
+      final ColumnIdentifier columnIdentifier, final String carbonStorePath) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.carbonStorePath = carbonStorePath;
@@ -117,9 +118,7 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
     boolean isNotNull =
         null != columnSortInfo.getSort_index() && null != columnSortInfo.sort_index_inverted;
     if (isNotNull) {
-      CarbonTablePath carbonTablePath =
-          CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
-      this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier);
+      initPath();
       String folderContainingFile = CarbonTablePath.getFolderContainingFile(this.sortIndexFilePath);
       boolean created = CarbonUtil.checkAndCreateFolder(folderContainingFile);
       if (!created) {
@@ -144,6 +143,12 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
     }
   }
 
+  protected void initPath() {
+    CarbonTablePath carbonTablePath =
+        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
+    this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier.getColumnId());
+  }
+
   /**
    * Closes this stream and releases any system resources associated
    * with it. If the stream is already closed then invoking this

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
index 1d68872..84cb975 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
@@ -357,16 +357,16 @@ public class QueryUtil {
         CarbonMetadata.getInstance().getCarbonTable(carbonTableIdentifier.getTableUniqueName());
     List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
         new ArrayList<>(dictionaryColumnIdList.size());
-    for (String columnIdentifier : dictionaryColumnIdList) {
+    for (String columnId : dictionaryColumnIdList) {
       CarbonDimension dimension = CarbonMetadata.getInstance()
-          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnIdentifier);
+          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
       if (null == dimension) {
         throw new QueryExecutionException(
-            "The column id " + columnIdentifier + " could not be resolved.");
+            "The column id " + columnId + " could not be resolved.");
       }
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
-              dimension.getDataType());
+          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+              dimension.getColumnIdentifier(), dimension.getDataType());
       dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
     }
     return dictionaryColumnUniqueIdentifiers;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java b/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
index 74cb4cd..810cbaf 100644
--- a/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
+++ b/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
@@ -871,7 +871,7 @@ public final class FilterUtil {
       CarbonDimension carbonDimension) throws QueryExecutionException {
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(tableIdentifier.getCarbonTableIdentifier(),
-            String.valueOf(carbonDimension.getColumnId()), carbonDimension.getDataType());
+            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType());
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache forwardDictionaryCache =
         cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, tableIdentifier.getStorePath());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java b/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
index be9566f..3a23d0d 100644
--- a/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
+++ b/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
@@ -14,6 +14,7 @@ import java.util.Properties;
 
 import org.carbondata.core.cache.Cache;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
@@ -84,7 +85,8 @@ public class AbstractDictionaryCacheTest {
   }
 
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
-      String columnIdentifier) {
+      String columnId) {
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataType.STRING);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
             DataType.STRING);
@@ -106,8 +108,9 @@ public class AbstractDictionaryCacheTest {
    * @param data
    * @throws IOException
    */
-  protected void prepareWriterAndWriteData(List<String> data, String columnIdentifier)
+  protected void prepareWriterAndWriteData(List<String> data, String columnId)
       throws IOException {
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     CarbonDictionaryWriter carbonDictionaryWriter =
         new CarbonDictionaryWriterImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
     CarbonTablePath carbonTablePath =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java b/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
index fab0e97..ad6787b 100644
--- a/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
+++ b/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
@@ -31,6 +31,7 @@ import org.carbondata.core.cache.Cache;
 import org.carbondata.core.cache.CacheProvider;
 import org.carbondata.core.cache.CacheType;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
@@ -208,7 +209,8 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
    * @param columnIdentifier
    * @throws IOException
    */
-  private void writeSortIndexFile(List<String> data, String columnIdentifier) throws IOException {
+  private void writeSortIndexFile(List<String> data, String columnId) throws IOException {
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     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/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
index ef8f43d..187e5e4 100644
--- a/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ b/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
@@ -32,6 +32,7 @@ import org.carbondata.core.cache.Cache;
 import org.carbondata.core.cache.CacheProvider;
 import org.carbondata.core.cache.CacheType;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.carbondata.core.util.CarbonProperties;
@@ -264,7 +265,8 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     }
   }
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
-	      String columnIdentifier) {
+	      String columnId) {
+	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
 	    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
 	        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
 	    return dictionaryColumnUniqueIdentifier;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java b/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
index 64767e1..8b10c32 100644
--- a/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
+++ b/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
@@ -58,8 +59,9 @@ public class CarbonDictionarySortIndexReaderImplTest {
     deleteStorePath();
     CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("testSchema", "carbon",
     		UUID.randomUUID().toString());
+    ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
     CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, "Name", hdfsStorePath);
+        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, columnIdentifier, hdfsStorePath);
     List<int[]> expectedData = prepareExpectedData();
 
     List<Integer> sortIndex = Arrays.asList(ArrayUtils.toObject(expectedData.get(0)));
@@ -68,7 +70,7 @@ public class CarbonDictionarySortIndexReaderImplTest {
     dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
     dictionarySortIndexWriter.close();
     CarbonDictionarySortIndexReader dictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, "Name", hdfsStorePath);
+        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier, hdfsStorePath);
     List<Integer> actualSortIndex = dictionarySortIndexReader.readSortIndex();
     List<Integer> actualInvertedSortIndex = dictionarySortIndexReader.readInvertedSortIndex();
     for (int i = 0; i < actualSortIndex.size(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java b/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
index 6c03636..31822d1 100644
--- a/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
+++ b/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
@@ -33,6 +33,7 @@ import java.util.Properties;
 import java.util.UUID;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -71,7 +72,7 @@ public class CarbonDictionaryWriterImplTest {
 
   private String carbonStorePath;
 
-  private String columnIdentifier;
+  private ColumnIdentifier columnIdentifier;
 
   private Properties props;
 
@@ -96,7 +97,7 @@ public class CarbonDictionaryWriterImplTest {
     this.databaseName = props.getProperty("database", "testSchema");
     this.tableName = props.getProperty("tableName", "carbon");
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    this.columnIdentifier = "Name";
+    this.columnIdentifier = new ColumnIdentifier("Name", null, null);
     carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
     deleteStorePath();
     prepareDataSet();
@@ -514,7 +515,7 @@ public class CarbonDictionaryWriterImplTest {
     if(!FileFactory.isFileExist(dictionaryLocation, fileType)) {
       FileFactory.mkdirs(dictionaryLocation, fileType);
     }
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier);
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier);
+    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId());
+    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
   }
 }