[CARBONATA-3605] Remove global dictionary feature

This closes #3502
diff --git a/conf/carbon.properties.template b/conf/carbon.properties.template
index 1709aac..1d5331c 100644
--- a/conf/carbon.properties.template
+++ b/conf/carbon.properties.template
@@ -85,7 +85,6 @@
 ##Min max feature is added to enhance query performance. To disable this feature, make it false.
 #carbon.enableMinMax=true
 
-######## Global Dictionary Configurations ########
 ##The property to set the date to be considered as start date for calculating the timestamp.
 #carbon.cutOffTimestamp
 ##The property to set the timestamp (ie milis) conversion to the SECOND, MINUTE, HOUR or DAY level.
diff --git a/conf/dataload.properties.template b/conf/dataload.properties.template
index 3b582b2..1a165a8 100644
--- a/conf/dataload.properties.template
+++ b/conf/dataload.properties.template
@@ -35,10 +35,7 @@
 #csv comment character
 #commentchar=#
 
-#column dictionary list
-#columndict=
-
-#null value's serialization format  
+#null value's serialization format
 #serialization_null_format=\\N
 
 #bad records logger
@@ -47,9 +44,6 @@
 #bad records action
 #bad_records_action=force
 
-#all dictionary folder path
-#all_dictionary_path=
-
 #complex column's level 1 delimiter
 #complex_delimiter_level_1='\\\001'
 
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
index deb48e2..e17641f 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
@@ -22,10 +22,6 @@
 import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.cache.dictionary.ForwardDictionaryCache;
-import org.apache.carbondata.core.cache.dictionary.ReverseDictionaryCache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.indexstore.BlockletDataMapIndexStore;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -89,13 +85,13 @@
   public <K, V> Cache<K, V> createCache(CacheType cacheType) {
     //check if lru cache is null, if null create one
     //check if cache is null for given cache type, if null create one
-    if (!dictionaryCacheAlreadyExists(cacheType)) {
+    if (!isCacheExists(cacheType)) {
       synchronized (lock) {
-        if (!dictionaryCacheAlreadyExists(cacheType)) {
+        if (!isCacheExists(cacheType)) {
           if (null == carbonLRUCache) {
             createLRULevelCacheInstance();
           }
-          createDictionaryCacheForGivenType(cacheType);
+          createBlockletDataMapCache(cacheType);
         }
       }
     }
@@ -110,9 +106,9 @@
       throws Exception {
     //check if lru cache is null, if null create one
     //check if cache is null for given cache type, if null create one
-    if (!dictionaryCacheAlreadyExists(cacheType)) {
+    if (!isCacheExists(cacheType)) {
       synchronized (lock) {
-        if (!dictionaryCacheAlreadyExists(cacheType)) {
+        if (!isCacheExists(cacheType)) {
           if (null == carbonLRUCache) {
             createLRULevelCacheInstance();
           }
@@ -132,14 +128,14 @@
    *
    * @param cacheType       type of cache
    */
-  private void createDictionaryCacheForGivenType(CacheType cacheType) {
+  private void createBlockletDataMapCache(CacheType cacheType) {
     Cache cacheObject = null;
     if (cacheType.equals(CacheType.REVERSE_DICTIONARY)) {
-      cacheObject =
-          new ReverseDictionaryCache<DictionaryColumnUniqueIdentifier, Dictionary>(carbonLRUCache);
+      // TODO: remove cache for dictionary after dictionary id deprecated
+      return;
     } else if (cacheType.equals(CacheType.FORWARD_DICTIONARY)) {
-      cacheObject =
-          new ForwardDictionaryCache<DictionaryColumnUniqueIdentifier, Dictionary>(carbonLRUCache);
+      // TODO: remove cache for dictionary after dictionary id deprecated
+      return;
     } else if (cacheType.equals(cacheType.DRIVER_BLOCKLET_DATAMAP)) {
       cacheObject = new BlockletDataMapIndexStore(carbonLRUCache);
     }
@@ -181,21 +177,10 @@
    * @param cacheType
    * @return
    */
-  private boolean dictionaryCacheAlreadyExists(CacheType cacheType) {
+  private boolean isCacheExists(CacheType cacheType) {
     return null != cacheTypeToCacheMap.get(cacheType);
   }
 
-  /**
-   * Below method will be used to clear the cache
-   */
-  public void dropAllCache() {
-    if (null != carbonLRUCache) {
-      carbonLRUCache.clear();
-      carbonLRUCache = null;
-    }
-    cacheTypeToCacheMap.clear();
-  }
-
   public CarbonLRUCache getCarbonCache() {
     return carbonLRUCache;
   }
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
index 8010d1c..b9c9e74 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.cache.dictionary;
 
-import java.nio.charset.Charset;
 import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -51,21 +50,11 @@
   protected long fileTimeStamp;
 
   /**
-   * offset till where file is read
-   */
-  protected long offsetTillFileIsRead;
-
-  /**
    * memory size of this object.We store it as calculation everytime is costly
    */
   protected long memorySize;
 
   /**
-   * length of dictionary metadata file
-   */
-  private long dictionaryMetaFileLength;
-
-  /**
    * size of one dictionary bucket
    */
   private final int dictionaryOneChunkSize = CarbonUtil.getDictionaryChunkSize();
@@ -102,31 +91,6 @@
     return memorySize;
   }
 
-  @Override
-  public void setMemorySize(long memorySize) {
-    this.memorySize = memorySize;
-  }
-
-  /**
-   * This method will increment the access count for a column by 1
-   * whenever a column is getting used in query or incremental data load
-   */
-  @Override
-  public void incrementAccessCount() {
-    accessCount.incrementAndGet();
-  }
-
-  /**
-   * This method will return the size of of last dictionary chunk so that only that many
-   * values are read from the dictionary reader
-   *
-   * @return size of last dictionary chunk
-   */
-  @Override
-  public int getSizeOfLastDictionaryChunk() {
-    return 0;
-  }
-
   /**
    * This method will decrement the access count for a column by 1
    * whenever a column usage is complete
@@ -138,32 +102,6 @@
   }
 
   /**
-   * This method will update the end offset of file everytime a file is read
-   *
-   * @param offsetTillFileIsRead
-   */
-  @Override
-  public void setOffsetTillFileIsRead(long offsetTillFileIsRead) {
-    this.offsetTillFileIsRead = offsetTillFileIsRead;
-  }
-
-  @Override
-  public long getOffsetTillFileIsRead() {
-    return offsetTillFileIsRead;
-  }
-
-  /**
-   * This method will update the timestamp of a file if a file is modified
-   * like in case of incremental load
-   *
-   * @param fileTimeStamp
-   */
-  @Override
-  public void setFileTimeStamp(long fileTimeStamp) {
-    this.fileTimeStamp = fileTimeStamp;
-  }
-
-  /**
    * The method return the list of dictionary chunks of a column
    * Applications Scenario.
    * For preparing the column Sort info while writing the sort index file.
@@ -184,56 +122,6 @@
   }
 
   /**
-   * This method will find and return the sort index for a given dictionary id.
-   * Applicable scenarios:
-   * 1. Used in case of order by queries when data sorting is required
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return if found returns key else 0
-   */
-  @Override
-  public int getSortedIndex(int surrogateKey) {
-    return 0;
-  }
-
-  /**
-   * dictionary metadata file length which will be set whenever we reload dictionary
-   * data from disk
-   *
-   * @param dictionaryMetaFileLength length of dictionary metadata file
-   */
-  @Override
-  public void setDictionaryMetaFileLength(long dictionaryMetaFileLength) {
-    this.dictionaryMetaFileLength = dictionaryMetaFileLength;
-  }
-
-  /**
-   * Dictionary meta file offset which will be read to check whether length of dictionary
-   * meta file has been modified
-   *
-   * @return
-   */
-  @Override
-  public long getDictionaryMetaFileLength() {
-    return dictionaryMetaFileLength;
-  }
-
-  /**
-   * This method will find and return the dictionary value from sorted index.
-   * Applicable scenarios:
-   * 1. Query final result preparation in case of order by queries:
-   * While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param sortedIndex sort index of dictionary value
-   * @return value if found else null
-   */
-  @Override
-  public String getDictionaryValueFromSortedIndex(int sortedIndex) {
-    return null;
-  }
-
-  /**
    * This method will find and return the dictionary value for a given surrogate key.
    * Applicable scenarios:
    * 1. Query final result preparation : While convert the final result which will
@@ -299,22 +187,6 @@
     return dictionaryValueInBytes;
   }
 
-  /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value
-   * @return if found returns key else INVALID_SURROGATE_KEY
-   */
-  @Override
-  public int getSurrogateKey(String value) {
-    byte[] keyData = value.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    return getSurrogateKey(keyData);
-  }
-
   @Override
   public void invalidate() {
 
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
deleted file mode 100644
index 7eef6b7..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.CarbonLRUCache;
-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.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.util.CarbonProperties;
-import org.apache.carbondata.core.util.ObjectSizeCalculator;
-
-/**
- * Abstract class which implements methods common to reverse and forward dictionary cache
- */
-public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
-    V extends Dictionary>
-    implements Cache<DictionaryColumnUniqueIdentifier, Dictionary> {
-
-  /**
-   * thread pool size to be used for dictionary data reading
-   */
-  protected int threadPoolSize;
-
-  /**
-   * LRU cache variable
-   */
-  protected CarbonLRUCache carbonLRUCache;
-
-  /**
-   * @param carbonLRUCache
-   */
-  public AbstractDictionaryCache(CarbonLRUCache carbonLRUCache) {
-    this.carbonLRUCache = carbonLRUCache;
-    initThreadPoolSize();
-  }
-
-  @Override
-  public void put(DictionaryColumnUniqueIdentifier key, Dictionary value) {
-    throw new UnsupportedOperationException("Operation not supported");
-  }
-
-  /**
-   * This method will initialize the thread pool size to be used for creating the
-   * max number of threads for a job
-   */
-  private void initThreadPoolSize() {
-    threadPoolSize = CarbonProperties.getInstance().getNumberOfLoadingCores();
-  }
-
-  /**
-   * This method will read dictionary metadata file and return the dictionary meta chunks
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return list of dictionary metadata chunks
-   * @throws IOException read and close method throws IO exception
-   */
-  protected CarbonDictionaryColumnMetaChunk readLastChunkFromDictionaryMetadataFile(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
-    CarbonDictionaryMetadataReader columnMetadataReaderImpl = dictService
-        .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier);
-
-    CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
-    // read metadata file
-    try {
-      carbonDictionaryColumnMetaChunk =
-          columnMetadataReaderImpl.readLastEntryOfDictionaryMetaChunk();
-    } finally {
-      // close the metadata reader
-      columnMetadataReaderImpl.close();
-    }
-    return carbonDictionaryColumnMetaChunk;
-  }
-
-  /**
-   * get the dictionary column meta chunk for object already read and stored in LRU cache
-   * @param dictionaryColumnUniqueIdentifier
-   * @param offsetRead
-   * @return
-   * @throws IOException
-   */
-  protected long getNumRecordsInCarbonDictionaryColumnMetaChunk(
-          DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, long offsetRead)
-          throws IOException {
-    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
-    CarbonDictionaryMetadataReader columnMetadataReaderImpl = dictService
-            .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier);
-
-    CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
-    // read metadata file
-    try {
-      carbonDictionaryColumnMetaChunk =
-              columnMetadataReaderImpl.readEntryOfDictionaryMetaChunk(offsetRead);
-    } finally {
-      // close the metadata reader
-      columnMetadataReaderImpl.close();
-    }
-    return carbonDictionaryColumnMetaChunk.getMax_surrogate_key();
-  }
-
-  /**
-   * This method will validate dictionary metadata file for any modification
-   *
-   * @param carbonFile
-   * @param fileTimeStamp
-   * @param endOffset
-   * @return
-   */
-  private boolean isDictionaryMetaFileModified(CarbonFile carbonFile, long fileTimeStamp,
-      long endOffset) {
-    return carbonFile.isFileModified(fileTimeStamp, endOffset);
-  }
-
-  /**
-   * This method will return the carbon file objetc based on its type (local, HDFS)
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  private CarbonFile getDictionaryMetaCarbonFile(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    String dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
-    CarbonFile dictFile = FileFactory.getCarbonFile(dictionaryFilePath);
-    // When rename table triggered parallely with select query, dictionary files may not exist
-    if (!dictFile.exists()) {
-      throw new IOException("Dictionary file does not exist: " + dictionaryFilePath);
-    }
-    return dictFile;
-  }
-
-  protected long getSortIndexSize(long numOfRecords) {
-    // sort index has sort index and reverse sort index,each is 4 byte integer.
-    // 32 byte is the array header of both the integer arrays
-    return numOfRecords * ObjectSizeCalculator.estimate(0, 16) * 2 + 32;
-  }
-
-  /**
-   * This method will get the value for the given key. If value does not exist
-   * for the given key, it will check and load the value.
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @param dictionaryInfo
-   * @param lruCacheKey
-   * @throws IOException                    in case memory is not sufficient to load dictionary
-   *                                        into memory
-   */
-  protected void checkAndLoadDictionaryData(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier,
-      DictionaryInfo dictionaryInfo, String lruCacheKey, boolean loadSortIndex)
-      throws IOException {
-    // read last segm
-    // ent dictionary meta chunk entry to get the end offset of file
-    CarbonFile carbonFile = getDictionaryMetaCarbonFile(dictionaryColumnUniqueIdentifier);
-    boolean dictionaryMetaFileModified =
-        isDictionaryMetaFileModified(carbonFile, dictionaryInfo.getFileTimeStamp(),
-            dictionaryInfo.getDictionaryMetaFileLength());
-    // if dictionary metadata file is modified then only read the last entry from dictionary
-    // meta file
-    if (dictionaryMetaFileModified) {
-      synchronized (dictionaryInfo) {
-        carbonFile = getDictionaryMetaCarbonFile(dictionaryColumnUniqueIdentifier);
-        dictionaryMetaFileModified =
-            isDictionaryMetaFileModified(carbonFile, dictionaryInfo.getFileTimeStamp(),
-                dictionaryInfo.getDictionaryMetaFileLength());
-        // Double Check :
-        // if dictionary metadata file is modified then only read the last entry from dictionary
-        // meta file
-        if (dictionaryMetaFileModified) {
-          CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk =
-              readLastChunkFromDictionaryMetadataFile(dictionaryColumnUniqueIdentifier);
-
-          long requiredSize = getEstimatedDictionarySize(dictionaryInfo,
-              carbonDictionaryColumnMetaChunk,
-              dictionaryColumnUniqueIdentifier, loadSortIndex);
-
-          if (requiredSize > 0) {
-            dictionaryInfo.setMemorySize(requiredSize);
-            boolean colCanBeAddedToLRUCache =
-                    carbonLRUCache.tryPut(lruCacheKey, requiredSize);
-            // if column can be added to lru cache then only load the
-            // dictionary data
-            if (colCanBeAddedToLRUCache) {
-              // load dictionary data
-              loadDictionaryData(dictionaryInfo, dictionaryColumnUniqueIdentifier,
-                      dictionaryInfo.getOffsetTillFileIsRead(),
-                      carbonDictionaryColumnMetaChunk.getEnd_offset(),
-                      loadSortIndex);
-              // set the end offset till where file is read
-              dictionaryInfo
-                      .setOffsetTillFileIsRead(carbonDictionaryColumnMetaChunk.getEnd_offset());
-              long updateRequiredSize = ObjectSizeCalculator.estimate(dictionaryInfo, requiredSize);
-              dictionaryInfo.setMemorySize(updateRequiredSize);
-              if (!carbonLRUCache.put(lruCacheKey, dictionaryInfo, updateRequiredSize)) {
-                throw new DictionaryBuilderException(
-                        "Cannot load dictionary into memory. Not enough memory available");
-              }
-              dictionaryInfo.setFileTimeStamp(carbonFile.getLastModifiedTime());
-              dictionaryInfo.setDictionaryMetaFileLength(carbonFile.getSize());
-            } else {
-              throw new DictionaryBuilderException(
-                      "Cannot load dictionary into memory. Not enough memory available");
-            }
-          }
-        }
-      }
-    }
-    // increment the column access count
-    incrementDictionaryAccessCount(dictionaryInfo);
-  }
-
-  /**
-   * This method will prepare the lru cache key and return the same
-   *
-   * @param columnIdentifier
-   * @return
-   */
-  protected String getLruCacheKey(String columnIdentifier, CacheType cacheType) {
-    return columnIdentifier + CarbonCommonConstants.UNDERSCORE + cacheType.getCacheName();
-  }
-
-  /**
-   * This method will check and load the dictionary file in memory for a given column
-   *
-   * @param dictionaryInfo                   holds dictionary information and data
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @param dictionaryChunkStartOffset       start offset from where dictionary file has to
-   *                                         be read
-   * @param dictionaryChunkEndOffset         end offset till where dictionary file has to
-   *                                         be read
-   * @param loadSortIndex
-   * @throws IOException
-   */
-  private void loadDictionaryData(DictionaryInfo dictionaryInfo,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier,
-      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
-      throws IOException {
-    DictionaryCacheLoader dictionaryCacheLoader =
-        new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
-    dictionaryCacheLoader
-        .load(dictionaryInfo, dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex);
-  }
-
-  /**
-   * This method will increment the access count for a given dictionary column
-   *
-   * @param dictionaryInfo
-   */
-  protected void incrementDictionaryAccessCount(DictionaryInfo dictionaryInfo) {
-    dictionaryInfo.incrementAccessCount();
-  }
-
-  /**
-   * This method will update the dictionary acceess count which is required for its removal
-   * from column LRU cache
-   *
-   * @param dictionaryList
-   */
-  protected void clearDictionary(List<Dictionary> dictionaryList) {
-    for (Dictionary dictionary : dictionaryList) {
-      dictionary.clear();
-    }
-  }
-
-  /**
-   * calculate the probable size of Dictionary in java heap
-   * Use the value to check if can be added to lru cache
-   * This helps to avoid unnecessary loading of dictionary files
-   * if estimated size more than that can be fit into lru cache
-   * Estimated size can be less or greater than the actual size
-   * due to java optimizations
-   * @param dictionaryInfo
-   * @param carbonDictionaryColumnMetaChunk
-   * @param dictionaryColumnUniqueIdentifier
-   * @param readSortIndexSize
-   * @return
-   * @throws IOException
-   */
-  protected long getEstimatedDictionarySize(DictionaryInfo dictionaryInfo,
-      CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, boolean
-      readSortIndexSize) throws IOException {
-    return 0;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
deleted file mode 100644
index e118816..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.format.ColumnDictionaryChunk;
-
-/**
- * This class is a wrapper over column dictionary chunk thrift object.
- * The wrapper class wraps the list<ColumnDictionaryChunk> and provides an API
- * to fill the byte array into list
- */
-public class ColumnDictionaryChunkIterator extends CarbonIterator {
-
-  /**
-   * list of dictionaryChunks
-   */
-  private List<ColumnDictionaryChunk> columnDictionaryChunks;
-
-  /**
-   * size of the list
-   */
-  private int size;
-
-  /**
-   * Current index of the list
-   */
-  private int currentSize;
-
-  /**
-   * variable holds the count of elements already iterated
-   */
-  private int iteratorIndex;
-
-  /**
-   * variable holds the current index of List<List<byte[]>> being traversed
-   */
-  private int outerIndex;
-
-  /**
-   * Constructor of ColumnDictionaryChunkIterator
-   *
-   * @param columnDictionaryChunks
-   */
-  public ColumnDictionaryChunkIterator(List<ColumnDictionaryChunk> columnDictionaryChunks) {
-    this.columnDictionaryChunks = columnDictionaryChunks;
-    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
-      this.size += dictionaryChunk.getValues().size();
-    }
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   * (In other words, returns {@code true} if {@link #next} would
-   * return an element rather than throwing an exception.)
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override
-  public boolean hasNext() {
-    return (currentSize < size);
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   * The method pics the next elements from the first inner list till first is not finished, pics
-   * the second inner list ...
-   *
-   * @return the next element in the iteration
-   */
-  @Override
-  public byte[] next() {
-    if (iteratorIndex >= columnDictionaryChunks.get(outerIndex).getValues().size()) {
-      iteratorIndex = 0;
-      outerIndex++;
-    }
-    ByteBuffer buffer = columnDictionaryChunks.get(outerIndex).getValues().get(iteratorIndex);
-    byte[] value = buffer.array();
-    currentSize++;
-    iteratorIndex++;
-    return value;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
index 08791a9..0b76b2e 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
@@ -26,7 +26,6 @@
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 /**
@@ -40,12 +39,6 @@
   private AtomicReference<List<Integer>> sortOrderReference =
       new AtomicReference<List<Integer>>(new ArrayList<Integer>());
 
-  /**
-   * inverted index to retrieve the member
-   */
-  private AtomicReference<List<Integer>> sortReverseOrderReference =
-      new AtomicReference<List<Integer>>(new ArrayList<Integer>());
-
   private DataType dataType;
 
   public ColumnDictionaryInfo(DataType dataType) {
@@ -53,176 +46,6 @@
   }
 
   /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value as byte array
-   * @return if found returns key else 0
-   */
-  @Override
-  public int getSurrogateKey(byte[] value) {
-    return getSurrogateKeyFromDictionaryValue(value);
-  }
-
-  /**
-   * This method will find and return the sort index for a given dictionary id.
-   * Applicable scenarios:
-   * 1. Used in case of order by queries when data sorting is required
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return if found returns key else 0
-   */
-  @Override
-  public int getSortedIndex(int surrogateKey) {
-    if (surrogateKey > sortReverseOrderReference.get().size()
-        || surrogateKey < MINIMUM_SURROGATE_KEY) {
-      return -1;
-    }
-    // decrement surrogate key as surrogate key basically means the index in array list
-    // because surrogate key starts from 1 and index of list from 0, so it needs to be
-    // decremented by 1
-    return sortReverseOrderReference.get().get(surrogateKey - 1);
-  }
-
-  /**
-   * This method will find and return the dictionary value from sorted index.
-   * Applicable scenarios:
-   * 1. Query final result preparation in case of order by queries:
-   * While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param sortedIndex sort index of dictionary value
-   * @return value if found else null
-   */
-  @Override
-  public String getDictionaryValueFromSortedIndex(int sortedIndex) {
-    if (sortedIndex > sortReverseOrderReference.get().size()
-        || sortedIndex < MINIMUM_SURROGATE_KEY) {
-      return null;
-    }
-    // decrement surrogate key as surrogate key basically means the index in array list
-    // because surrogate key starts from 1, sort index will start form 1 and index
-    // of list from 0, so it needs to be decremented by 1
-    int surrogateKey = sortOrderReference.get().get(sortedIndex - 1);
-    return getDictionaryValueForKey(surrogateKey);
-  }
-
-  /**
-   * This method will add a new dictionary chunk to existing list of dictionary chunks
-   *
-   * @param newDictionaryChunk
-   */
-  @Override
-  public void addDictionaryChunk(List<byte[]> newDictionaryChunk) {
-    if (dictionaryChunks.size() > 0) {
-      // Ensure that each time a new dictionary chunk is getting added to the
-      // dictionary chunks list, equal distribution of dictionary values should
-      // be there in the sublists of dictionary chunk list
-      List<byte[]> lastDictionaryChunk = dictionaryChunks.get(dictionaryChunks.size() - 1);
-      int dictionaryOneChunkSize = CarbonUtil.getDictionaryChunkSize();
-      int differenceInLastDictionaryAndOneChunkSize =
-          dictionaryOneChunkSize - lastDictionaryChunk.size();
-      if (differenceInLastDictionaryAndOneChunkSize > 0) {
-        // if difference is greater than new dictionary size then copy a part of list
-        // else copy the complete new dictionary chunk list in the last dictionary chunk list
-        if (differenceInLastDictionaryAndOneChunkSize >= newDictionaryChunk.size()) {
-          lastDictionaryChunk.addAll(newDictionaryChunk);
-        } else {
-          List<byte[]> subListOfNewDictionaryChunk =
-              newDictionaryChunk.subList(0, differenceInLastDictionaryAndOneChunkSize);
-          lastDictionaryChunk.addAll(subListOfNewDictionaryChunk);
-          List<byte[]> remainingNewDictionaryChunk = newDictionaryChunk
-              .subList(differenceInLastDictionaryAndOneChunkSize, newDictionaryChunk.size());
-          dictionaryChunks.add(remainingNewDictionaryChunk);
-        }
-      } else {
-        dictionaryChunks.add(newDictionaryChunk);
-      }
-    } else {
-      dictionaryChunks.add(newDictionaryChunk);
-    }
-  }
-
-  /**
-   * This method will return the size of of last dictionary chunk so that only that many
-   * values are read from the dictionary reader
-   *
-   * @return size of last dictionary chunk
-   */
-  @Override
-  public int getSizeOfLastDictionaryChunk() {
-    int lastDictionaryChunkSize = 0;
-    if (dictionaryChunks.size() > 0) {
-      lastDictionaryChunkSize = dictionaryChunks.get(dictionaryChunks.size() - 1).size();
-    }
-    return lastDictionaryChunkSize;
-  }
-
-  /**
-   * This method will set the sort order index of a dictionary column.
-   * Sort order index if the index of dictionary values after they are sorted.
-   *
-   * @param sortOrderIndex
-   */
-  @Override
-  public void setSortOrderIndex(List<Integer> sortOrderIndex) {
-    sortOrderReference.set(sortOrderIndex);
-  }
-
-  /**
-   * This method will set the sort reverse index of a dictionary column.
-   * Sort reverse index is the index of dictionary values before they are sorted.
-   *
-   * @param sortReverseOrderIndex
-   */
-  @Override
-  public void setSortReverseOrderIndex(List<Integer> sortReverseOrderIndex) {
-    sortReverseOrderReference.set(sortReverseOrderIndex);
-  }
-
-  /**
-   * This method will apply binary search logic to find the surrogate key for the
-   * given value
-   *
-   * @param key to be searched
-   * @return
-   */
-  private int getSurrogateKeyFromDictionaryValue(byte[] key) {
-    String filterKey = new String(key, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    int low = 0;
-    List<Integer> sortedSurrogates = sortOrderReference.get();
-    int high = sortedSurrogates.size() - 1;
-    while (low <= high) {
-      int mid = (low + high) >>> 1;
-      int surrogateKey = sortedSurrogates.get(mid);
-      byte[] dictionaryValue = getDictionaryBytesFromSurrogate(surrogateKey);
-      if (null == dictionaryValue) {
-        return CarbonCommonConstants.INVALID_SURROGATE_KEY;
-      }
-      int cmp = -1;
-      if (this.getDataType() != DataTypes.STRING) {
-        cmp = compareFilterKeyWithDictionaryKey(
-            new String(dictionaryValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
-            filterKey, this.getDataType());
-
-      } else {
-        cmp = ByteUtil.UnsafeComparer.INSTANCE.compareTo(dictionaryValue, key);
-      }
-      if (cmp < 0) {
-        low = mid + 1;
-      } else if (cmp > 0) {
-        high = mid - 1;
-      } else {
-        return surrogateKey; // key found
-      }
-    }
-    return CarbonCommonConstants.INVALID_SURROGATE_KEY;
-  }
-
-  /**
    * This method will apply binary search logic to find the surrogate key for the
    * given value
    *
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnReverseDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnReverseDictionaryInfo.java
deleted file mode 100644
index 84c1953..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnReverseDictionaryInfo.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import net.jpountz.xxhash.XXHash32;
-import net.jpountz.xxhash.XXHashFactory;
-
-/**
- * class that implements methods specific for dictionary data look up
- */
-public class ColumnReverseDictionaryInfo extends AbstractColumnDictionaryInfo {
-
-  /**
-   * Map which will maintain mapping of byte array to surrogate key
-   */
-  private Map<DictionaryByteArrayWrapper, Integer> dictionaryByteArrayToSurrogateKeyMap;
-
-  /**
-   * hashing algorithm to calculate hash code
-   */
-  private XXHash32 xxHash32;
-
-  /**
-   * check and initialize xxHash32 if enabled
-   */
-  public ColumnReverseDictionaryInfo() {
-    boolean useXXHash = Boolean.valueOf(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.ENABLE_XXHASH,
-            CarbonCommonConstants.ENABLE_XXHASH_DEFAULT));
-    if (useXXHash) {
-      xxHash32 = XXHashFactory.fastestInstance().hash32();
-    }
-  }
-
-  /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value as byte array. It will be treated as key here
-   * @return if found returns key else INVALID_SURROGATE_KEY
-   */
-  @Override
-  public int getSurrogateKey(byte[] value) {
-    DictionaryByteArrayWrapper dictionaryByteArrayWrapper =
-        new DictionaryByteArrayWrapper(value, xxHash32);
-    Integer surrogateKeyInMap =
-        dictionaryByteArrayToSurrogateKeyMap.get(dictionaryByteArrayWrapper);
-    if (null == surrogateKeyInMap) {
-      return CarbonCommonConstants.INVALID_SURROGATE_KEY;
-    }
-    return surrogateKeyInMap;
-  }
-
-  /**
-   * This method will add a new dictionary chunk to existing list of dictionary chunks
-   *
-   * @param dictionaryChunk
-   */
-  @Override
-  public void addDictionaryChunk(List<byte[]> dictionaryChunk) {
-    dictionaryChunks.add(dictionaryChunk);
-    if (null == dictionaryByteArrayToSurrogateKeyMap) {
-      createDictionaryByteArrayToSurrogateKeyMap(dictionaryChunk.size());
-    }
-    addDataToDictionaryMap();
-  }
-
-  /**
-   * This method will add the new dictionary data to map
-   */
-  private void addDataToDictionaryMap() {
-    int surrogateKey = dictionaryByteArrayToSurrogateKeyMap.size();
-    List<byte[]> oneDictionaryChunk = dictionaryChunks.get(dictionaryChunks.size() - 1);
-    for (int i = 0; i < oneDictionaryChunk.size(); i++) {
-      // create a wrapper class that will calculate hash code for byte array
-      DictionaryByteArrayWrapper dictionaryByteArrayWrapper =
-          new DictionaryByteArrayWrapper(oneDictionaryChunk.get(i), xxHash32);
-      dictionaryByteArrayToSurrogateKeyMap.put(dictionaryByteArrayWrapper, ++surrogateKey);
-    }
-  }
-
-  /**
-   * This method will create the dictionary map. First time it will
-   * create dictionary map with capacity equal to list of byte arrays
-   *
-   * @param initialMapSize capacity to which map is to be instantiated
-   */
-  private void createDictionaryByteArrayToSurrogateKeyMap(int initialMapSize) {
-    dictionaryByteArrayToSurrogateKeyMap = new ConcurrentHashMap<>(initialMapSize);
-  }
-
-  /**
-   * This method will set the sort order index of a dictionary column.
-   * Sort order index if the index of dictionary values after they are sorted.
-   *
-   * @param sortOrderIndex
-   */
-  @Override
-  public void setSortOrderIndex(List<Integer> sortOrderIndex) {
-  }
-
-  /**
-   * This method will set the sort reverse index of a dictionary column.
-   * Sort reverse index is the index of dictionary values before they are sorted.
-   *
-   * @param sortReverseOrderIndex
-   */
-  @Override
-  public void setSortReverseOrderIndex(List<Integer> sortReverseOrderIndex) {
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/Dictionary.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/Dictionary.java
index 232d5f5..e6fc53a 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/Dictionary.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/Dictionary.java
@@ -24,30 +24,6 @@
 public interface Dictionary {
 
   /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value
-   * @return if found returns key else 0
-   */
-  int getSurrogateKey(String value);
-
-  /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value as byte array
-   * @return if found returns key else -1
-   */
-  int getSurrogateKey(byte[] value);
-
-  /**
    * This method will find and return the dictionary value for a given surrogate key.
    * Applicable scenarios:
    * 1. Query final result preparation : While convert the final result which will
@@ -72,28 +48,6 @@
   byte[] getDictionaryValueForKeyInBytes(int surrogateKey);
 
   /**
-   * This method will find and return the sort index for a given dictionary id.
-   * Applicable scenarios:
-   * 1. Used in case of order by queries when data sorting is required
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return if found returns key else 0
-   */
-  int getSortedIndex(int surrogateKey);
-
-  /**
-   * This method will find and return the dictionary value from sorted index.
-   * Applicable scenarios:
-   * 1. Query final result preparation in case of order by queries:
-   * While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param sortedIndex sort index of dictionary value
-   * @return value if found else null
-   */
-  String getDictionaryValueFromSortedIndex(int sortedIndex);
-
-  /**
    * The method return the dictionary chunks wrapper of a column
    * The wrapper wraps the list<list<bye[]>> and provide the iterator to retrieve the chunks
    * members.
@@ -109,9 +63,4 @@
    */
   void clear();
 
-  /**
-   * This method return the access count associated with the dictionary.
-   * @return
-   */
-  int getAccessCount();
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryBuilderException.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryBuilderException.java
deleted file mode 100644
index ed6394e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryBuilderException.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-
-public class DictionaryBuilderException extends IOException {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 21312121L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param msg       The error message for this exception.
-   */
-  public DictionaryBuilderException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-}
-
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
deleted file mode 100644
index 8df1539..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-
-public interface DictionaryCacheLoader {
-
-  /**
-   * This method will load the dictionary data for a given columnIdentifier
-   *
-   * @param dictionaryInfo             dictionary info object which will hold the required data
-   *                                   for a given column
-   * @param dictionaryChunkStartOffset start offset from where dictionary file has to
-   *                                   be read
-   * @param dictionaryChunkEndOffset   end offset till where dictionary file has to
-   *                                   be read
-   * @param loadSortIndex              flag to indicate whether the sort index file has to be
-   *                                   read in memory after dictionary loading
-   * @throws IOException
-   */
-  void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset,
-      long dictionaryChunkEndOffset, boolean loadSortIndex)
-      throws IOException;
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
deleted file mode 100644
index 899abf5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.reader.CarbonDictionaryReader;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.DictionaryService;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-/**
- * This class is responsible for loading the dictionary data for given columns
- */
-public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
-
-  /**
-   * carbon dictionary column identifier
-   */
-  private DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  /**
-   * @param dictionaryColumnUniqueIdentifier dictionary column identifier
-   */
-  DictionaryCacheLoaderImpl(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-  }
-
-  /**
-   * This method will load the dictionary data for a given columnIdentifier
-   *
-   * @param dictionaryInfo             dictionary info object which will hold the required data
-   *                                   for a given column
-   * @param dictionaryChunkStartOffset start offset from where dictionary file has to
-   *                                   be read
-   * @param dictionaryChunkEndOffset   end offset till where dictionary file has to
-   *                                   be read
-   * @param loadSortIndex              flag to indicate whether the sort index file has to be
-   *                                   read in memory after dictionary loading
-   * @throws IOException
-   */
-  @Override
-  public void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset,
-      long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException {
-    Iterator<byte[]> columnDictionaryChunkWrapper =
-        load(dictionaryColumnUniqueIdentifier, dictionaryChunkStartOffset,
-            dictionaryChunkEndOffset);
-    if (loadSortIndex) {
-      readSortIndexFile(dictionaryInfo, dictionaryColumnUniqueIdentifier);
-    }
-    fillDictionaryValuesAndAddToDictionaryChunks(dictionaryInfo, columnDictionaryChunkWrapper);
-  }
-
-  /**
-   * This method will fill the dictionary values according to dictionary bucket size and
-   * add to the dictionary chunk list
-   *
-   * @param dictionaryInfo
-   * @param columnDictionaryChunkWrapper
-   */
-  private void fillDictionaryValuesAndAddToDictionaryChunks(DictionaryInfo dictionaryInfo,
-      Iterator<byte[]> columnDictionaryChunkWrapper) {
-    int dictionaryChunkSize = CarbonUtil.getDictionaryChunkSize();
-    int sizeOfLastDictionaryChunk = dictionaryInfo.getSizeOfLastDictionaryChunk();
-    int sizeOfOneDictionaryChunk = dictionaryChunkSize - sizeOfLastDictionaryChunk;
-    if (sizeOfOneDictionaryChunk <= 0) {
-      sizeOfOneDictionaryChunk = dictionaryChunkSize;
-    }
-    List<List<byte[]>> dictionaryChunks =
-        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<byte[]> oneDictionaryChunk = new ArrayList<>(sizeOfOneDictionaryChunk);
-    dictionaryChunks.add(oneDictionaryChunk);
-    while (columnDictionaryChunkWrapper.hasNext()) {
-      oneDictionaryChunk.add(columnDictionaryChunkWrapper.next());
-      if (oneDictionaryChunk.size() >= sizeOfOneDictionaryChunk) {
-        sizeOfOneDictionaryChunk = dictionaryChunkSize;
-        oneDictionaryChunk = new ArrayList<>(sizeOfOneDictionaryChunk);
-        dictionaryChunks.add(oneDictionaryChunk);
-      }
-    }
-    for (List<byte[]> dictionaryChunk : dictionaryChunks) {
-      if (!dictionaryChunk.isEmpty()) {
-        dictionaryInfo.addDictionaryChunk(dictionaryChunk);
-      }
-    }
-  }
-
-  /**
-   * This method will load the dictionary data between a given start and end offset
-   *
-   * @param dictionaryColumnUniqueIdentifier column unique identifier
-   * @param startOffset      start offset of dictionary file
-   * @param endOffset        end offset of dictionary file
-   * @return iterator over dictionary values
-   * @throws IOException
-   */
-  private Iterator<byte[]> load(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier,
-      long startOffset, long endOffset) throws IOException {
-    CarbonDictionaryReader dictionaryReader = getDictionaryReader(dictionaryColumnUniqueIdentifier);
-    try {
-      return dictionaryReader.read(startOffset, endOffset);
-    } finally {
-      dictionaryReader.close();
-    }
-  }
-
-  /**
-   * This method will read the sort index file and load into memory
-   *
-   * @param dictionaryInfo
-   * @param dictionaryColumnUniqueIdentifier
-   * @throws IOException
-   */
-  private void readSortIndexFile(DictionaryInfo dictionaryInfo,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    CarbonDictionarySortIndexReader sortIndexReader =
-        getSortIndexReader(dictionaryColumnUniqueIdentifier);
-    try {
-      dictionaryInfo.setSortOrderIndex(sortIndexReader.readSortIndex());
-      dictionaryInfo.setSortReverseOrderIndex(sortIndexReader.readInvertedSortIndex());
-    } finally {
-      sortIndexReader.close();
-    }
-  }
-
-  /**
-   * This method will create a dictionary reader instance to read the dictionary file
-   *
-   * @param dictionaryColumnUniqueIdentifier unique column identifier
-   * @return carbon dictionary reader instance
-   */
-  private CarbonDictionaryReader getDictionaryReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
-    return dictService.getDictionaryReader(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * @param dictionaryColumnUniqueIdentifier unique column identifier
-   * @return sort index reader instance
-   */
-  private CarbonDictionarySortIndexReader getSortIndexReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
-    return dictService
-        .getDictionarySortIndexReader(dictionaryColumnUniqueIdentifier);
-  }
-}
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 d525a53..45f94e7 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
@@ -22,7 +22,6 @@
 import 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.CarbonTablePath;
 
 /**
  * dictionary column identifier which includes table identifier and column identifier
@@ -43,8 +42,6 @@
 
   private DataType dataType;
 
-  private String dictionaryLocation;
-
   /**
    * Will be used in case of reverse dictionary cache which will be used
    * in case of data loading.
@@ -64,8 +61,6 @@
     this.dictionarySourceAbsoluteTableIdentifier = dictionarySourceAbsoluteTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.dataType = columnIdentifier.getDataType();
-    this.dictionaryLocation =
-        CarbonTablePath.getMetadataPath(dictionarySourceAbsoluteTableIdentifier.getTablePath());
   }
 
   /**
@@ -83,15 +78,6 @@
     this.dataType = dataType;
   }
 
-  public DictionaryColumnUniqueIdentifier(
-      AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier,
-      ColumnIdentifier columnIdentifier, DataType dataType, String dictionaryLocation) {
-    this(dictionarySourceAbsoluteTableIdentifier, columnIdentifier, dataType);
-    if (null != dictionaryLocation) {
-      this.dictionaryLocation = dictionaryLocation;
-    }
-  }
-
   public DataType getDataType() {
     return dataType;
   }
@@ -104,39 +90,6 @@
   }
 
   /**
-   * @return dictionary file path
-   */
-  public String getDictionaryFilePath() {
-    return CarbonTablePath.getExternalDictionaryFilePath(
-        dictionaryLocation, columnIdentifier.getColumnId());
-  }
-
-  /**
-   * @return dictionary metadata file path
-   */
-  public String getDictionaryMetaFilePath() {
-    return CarbonTablePath.getExternalDictionaryMetaFilePath(
-        dictionaryLocation, columnIdentifier.getColumnId());
-  }
-
-  /**
-   * @return sort index file path
-   */
-  public String getSortIndexFilePath() {
-    return CarbonTablePath.getExternalSortIndexFilePath(
-        dictionaryLocation, columnIdentifier.getColumnId());
-  }
-
-  /**
-   * @param offset
-   * @return sort index file path with given offset
-   */
-  public String getSortIndexFilePath(long offset) {
-    return CarbonTablePath.getExternalSortIndexFilePath(
-        dictionaryLocation, columnIdentifier.getColumnId(), offset);
-  }
-
-  /**
    * overridden equals method
    *
    * @param other
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
index 0fbb4bb..5faa24b 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
@@ -17,93 +17,11 @@
 
 package org.apache.carbondata.core.cache.dictionary;
 
-import java.util.List;
-
 import org.apache.carbondata.core.cache.Cacheable;
 
 /**
- * An interface which holds dictionary information like end offset,
- * file timestamp for one column
+ * Deprecated
  */
 public interface DictionaryInfo extends Cacheable, Dictionary {
 
-  /**
-   * This method will increment the access count for a column by 1
-   * whenever a column is getting used in query or incremental data load
-   */
-  void incrementAccessCount();
-
-  /**
-   * This method will update the end offset of file everytime a file is read
-   *
-   * @param offsetTillFileIsRead
-   */
-  void setOffsetTillFileIsRead(long offsetTillFileIsRead);
-
-  /**
-   * offset till the file is read
-   * @return
-   */
-  long getOffsetTillFileIsRead();
-
-  /**
-   * the memory size of this object after loaded into memory
-   * @param memorySize
-   */
-  void setMemorySize(long memorySize);
-
-  /**
-   * This method will update the timestamp of a file if a file is modified
-   * like in case of incremental load
-   *
-   * @param fileTimeStamp
-   */
-  void setFileTimeStamp(long fileTimeStamp);
-
-  /**
-   * This method will add a new dictionary chunk to existing list of dictionary chunks
-   *
-   * @param dictionaryChunk
-   */
-  void addDictionaryChunk(List<byte[]> dictionaryChunk);
-
-  /**
-   * This method will return the size of of last dictionary chunk so that only that many
-   * values are read from the dictionary reader
-   *
-   * @return size of last dictionary chunk
-   */
-  int getSizeOfLastDictionaryChunk();
-
-  /**
-   * This method will set the sort order index of a dictionary column.
-   * Sort order index if the index of dictionary values after they are sorted.
-   *
-   * @param sortOrderIndex
-   */
-  void setSortOrderIndex(List<Integer> sortOrderIndex);
-
-  /**
-   * This method will set the sort reverse index of a dictionary column.
-   * Sort reverse index is the index of dictionary values before they are sorted.
-   *
-   * @param sortReverseOrderIndex
-   */
-  void setSortReverseOrderIndex(List<Integer> sortReverseOrderIndex);
-
-  /**
-   * dictionary metadata file length which will be set whenever we reload dictionary
-   * data from disk
-   *
-   * @param dictionaryMetaFileLength length of dictionary metadata file
-   */
-  void setDictionaryMetaFileLength(long dictionaryMetaFileLength);
-
-  /**
-   * Dictionary meta file offset which will be read to check whether length of dictionary
-   * meta file has been modified
-   *
-   * @return
-   */
-  long getDictionaryMetaFileLength();
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionary.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionary.java
index 5e690fa..17325fb 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionary.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionary.java
@@ -41,36 +41,6 @@
   }
 
   /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value
-   * @return if found returns key else INVALID_SURROGATE_KEY
-   */
-  @Override
-  public int getSurrogateKey(String value) {
-    return columnDictionaryInfo.getSurrogateKey(value);
-  }
-
-  /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value as byte array
-   * @return if found returns key else INVALID_SURROGATE_KEY
-   */
-  @Override
-  public int getSurrogateKey(byte[] value) {
-    return columnDictionaryInfo.getSurrogateKey(value);
-  }
-
-  /**
    * This method will find and return the dictionary value for a given surrogate key.
    * Applicable scenarios:
    * 1. Query final result preparation : While convert the final result which will
@@ -99,34 +69,6 @@
   }
 
   /**
-   * This method will find and return the sort index for a given dictionary id.
-   * Applicable scenarios:
-   * 1. Used in case of order by queries when data sorting is required
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return if found returns key else 0
-   */
-  @Override
-  public int getSortedIndex(int surrogateKey) {
-    return columnDictionaryInfo.getSortedIndex(surrogateKey);
-  }
-
-  /**
-   * This method will find and return the dictionary value from sorted index.
-   * Applicable scenarios:
-   * 1. Query final result preparation in case of order by queries:
-   * While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param sortedIndex sort index of dictionary value
-   * @return value if found else null
-   */
-  @Override
-  public String getDictionaryValueFromSortedIndex(int sortedIndex) {
-    return columnDictionaryInfo.getDictionaryValueFromSortedIndex(sortedIndex);
-  }
-
-  /**
    * The method return the dictionary chunks wrapper of a column
    * The wrapper wraps the list<list<bye[]>> and provide the iterator to retrieve the chunks
    * members.
@@ -169,16 +111,4 @@
     columnDictionaryInfo
         .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
   }
-
-  /**
-   * This method return the access count associated with the dictionary.
-   * @return
-   */
-  @Override
-  public int getAccessCount() {
-    if (null != columnDictionaryInfo) {
-      return columnDictionaryInfo.getAccessCount();
-    }
-    return 0;
-  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
deleted file mode 100644
index 9698e6c..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.CarbonLRUCache;
-import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.ObjectSizeCalculator;
-import org.apache.carbondata.core.util.TaskMetricsMap;
-
-import org.apache.log4j.Logger;
-
-/**
- * This class implements methods to create dictionary cache which will hold
- * dictionary chunks for look up of surrogate keys and values
- */
-public class ForwardDictionaryCache<K extends
-    DictionaryColumnUniqueIdentifier, V extends Dictionary> extends AbstractDictionaryCache<K, V> {
-
-  /**
-   * Attribute for Carbon LOGGER
-   */
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(ForwardDictionaryCache.class.getName());
-
-  private static final Map<DictionaryColumnUniqueIdentifier, Object> DICTIONARY_LOCK_OBJECT =
-      new ConcurrentHashMap<>();
-
-  private static final long sizeOfEmptyDictChunks =
-      ObjectSizeCalculator.estimate(new ArrayList<byte[]>(CarbonUtil.getDictionaryChunkSize()), 16);
-
-  private static final long byteArraySize = ObjectSizeCalculator.estimate(new byte[0], 16);
-
-  /**
-   * @param carbonLRUCache
-   */
-  public ForwardDictionaryCache(CarbonLRUCache carbonLRUCache) {
-    super(carbonLRUCache);
-  }
-
-  /**
-   * This method will get the value for the given key. If value does not exist
-   * for the given key, it will check and load the value.
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return dictionary
-   * @throws IOException in case memory is not sufficient to load dictionary into memory
-   */
-  @Override
-  public Dictionary get(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier)
-      throws IOException {
-    return getDictionary(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * This method will return a list of values for the given list of keys.
-   * For each key, this method will check and load the data if required.
-   *
-   * @param dictionaryColumnUniqueIdentifiers unique identifier which contains dbName,
-   *                                          tableName and columnIdentifier
-   * @return list of dictionary
-   * @throws IOException in case memory is not sufficient to load dictionary into memory
-   */
-  @Override
-  public List<Dictionary> getAll(
-      List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers) throws IOException {
-    boolean exceptionOccurredInDictionaryLoading = false;
-    String exceptionMessage = "";
-    List<Dictionary> forwardDictionaryObjectList =
-        new ArrayList<Dictionary>(dictionaryColumnUniqueIdentifiers.size());
-    List<Future<Dictionary>> taskSubmitList =
-        new ArrayList<>(dictionaryColumnUniqueIdentifiers.size());
-    ExecutorService executorService = Executors.newFixedThreadPool(threadPoolSize);
-    for (final DictionaryColumnUniqueIdentifier uniqueIdent : dictionaryColumnUniqueIdentifiers) {
-      taskSubmitList.add(executorService.submit(new Callable<Dictionary>() {
-        @Override
-        public Dictionary call() throws IOException {
-          try {
-            // Register thread callback for calculating metrics
-            TaskMetricsMap.getInstance().registerThreadCallback();
-            // in case of multiple task for same query same executor
-            // only one task should load the dictionary
-            // others will wait on monitor and get the loaded dictionary values
-            Object lockObject = DICTIONARY_LOCK_OBJECT.get(uniqueIdent);
-            // if lock object is null
-            if (null == lockObject) {
-              // Acquire the lock on map
-              synchronized (DICTIONARY_LOCK_OBJECT) {
-                // double checking the dictionary lock object
-                lockObject = DICTIONARY_LOCK_OBJECT.get(uniqueIdent);
-                // if still it is null add new lock object
-                if (null == lockObject) {
-                  lockObject = new Object();
-                  DICTIONARY_LOCK_OBJECT.put(uniqueIdent, lockObject);
-                }
-              }
-            }
-            Dictionary dictionary = null;
-            synchronized (lockObject) {
-              dictionary = getDictionary(uniqueIdent);
-            }
-            return dictionary;
-          }  finally {
-            // update read bytes metrics for this thread
-            TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId());
-          }
-
-        }
-      }));
-    }
-    try {
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.HOURS);
-    } catch (InterruptedException e) {
-      LOGGER.error("Error loading the dictionary: " + e.getMessage(), e);
-    }
-    for (int i = 0; i < taskSubmitList.size(); i++) {
-      try {
-        Dictionary columnDictionary = taskSubmitList.get(i).get();
-        forwardDictionaryObjectList.add(columnDictionary);
-      } catch (Throwable e) {
-        exceptionOccurredInDictionaryLoading = true;
-        exceptionMessage = e.getMessage();
-      }
-    }
-    if (exceptionOccurredInDictionaryLoading) {
-      clearDictionary(forwardDictionaryObjectList);
-      LOGGER.error(exceptionMessage);
-      throw new IOException(exceptionMessage);
-    }
-    return forwardDictionaryObjectList;
-  }
-
-  /**
-   * This method will return the value for the given key. It will not check and load
-   * the data for the given key
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return
-   */
-  @Override
-  public Dictionary getIfPresent(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    Dictionary forwardDictionary = null;
-    ColumnDictionaryInfo columnDictionaryInfo = (ColumnDictionaryInfo) carbonLRUCache.get(
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-            CacheType.FORWARD_DICTIONARY));
-    if (null != columnDictionaryInfo) {
-      forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
-      incrementDictionaryAccessCount(columnDictionaryInfo);
-    }
-    return forwardDictionary;
-  }
-
-  /**
-   * This method will remove the cache for a given key
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   */
-  @Override
-  public void invalidate(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    carbonLRUCache.remove(
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-            CacheType.FORWARD_DICTIONARY));
-  }
-
-  /**
-   * This method will get the value for the given key. If value does not exist
-   * for the given key, it will check and load the value.
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return dictionary
-   * @throws IOException in case memory is not sufficient to load dictionary into memory
-   */
-  private Dictionary getDictionary(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    Dictionary forwardDictionary = null;
-    // dictionary is only for primitive data type
-    assert (!dictionaryColumnUniqueIdentifier.getDataType().isComplexType());
-    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().getColumnId(),
-            CacheType.FORWARD_DICTIONARY), true);
-    forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
-    return forwardDictionary;
-  }
-
-  /**
-   * This method will check and create columnDictionaryInfo object for the given column
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @param columnIdentifier
-   * @return
-   */
-  private ColumnDictionaryInfo getColumnDictionaryInfo(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String columnIdentifier) {
-    ColumnDictionaryInfo columnDictionaryInfo = (ColumnDictionaryInfo) carbonLRUCache
-        .get(getLruCacheKey(columnIdentifier, CacheType.FORWARD_DICTIONARY));
-    if (null == columnDictionaryInfo) {
-      synchronized (dictionaryColumnUniqueIdentifier) {
-        columnDictionaryInfo = (ColumnDictionaryInfo) carbonLRUCache
-            .get(getLruCacheKey(columnIdentifier, CacheType.FORWARD_DICTIONARY));
-        if (null == columnDictionaryInfo) {
-          columnDictionaryInfo =
-              new ColumnDictionaryInfo(dictionaryColumnUniqueIdentifier.getDataType());
-        }
-      }
-    }
-    return columnDictionaryInfo;
-  }
-
-  @Override
-  public void clearAccessCount(List<DictionaryColumnUniqueIdentifier> keys) {
-    for (DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier : keys) {
-      Dictionary cacheable = (Dictionary) carbonLRUCache.get(
-          getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-              CacheType.FORWARD_DICTIONARY));
-      cacheable.clear();
-    }
-  }
-
-  @Override
-  protected long getEstimatedDictionarySize(DictionaryInfo dictionaryInfo,
-      CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, boolean
-      readSortIndexSize) throws IOException {
-    // required size will be size total size of file - offset till file is
-    // already read
-    long requiredSize =
-        carbonDictionaryColumnMetaChunk.getEnd_offset() -
-            dictionaryInfo.getOffsetTillFileIsRead();
-
-    long numOfRecords = dictionaryInfo.getOffsetTillFileIsRead() == 0 ?
-        carbonDictionaryColumnMetaChunk.getMax_surrogate_key() :
-        carbonDictionaryColumnMetaChunk.getMax_surrogate_key()
-            - getNumRecordsInCarbonDictionaryColumnMetaChunk(
-            dictionaryColumnUniqueIdentifier,
-            dictionaryInfo.getOffsetTillFileIsRead());
-
-    if (numOfRecords > 0) {
-      long avgRecordsSize = requiredSize / numOfRecords;
-      long bytesPerRecord = (long)Math.ceil(avgRecordsSize / 8.0) * 8;
-
-      requiredSize = (bytesPerRecord + byteArraySize) * numOfRecords;
-    }
-
-    if (readSortIndexSize) {
-      // every time we are loading all the sort index files.Hence memory calculation for all
-      // the records
-      requiredSize = requiredSize + getSortIndexSize(
-          carbonDictionaryColumnMetaChunk.getMax_surrogate_key());
-    }
-
-    return requiredSize + sizeOfEmptyDictChunks;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
deleted file mode 100644
index 98ae012..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-import org.apache.log4j.Logger;
-
-/**
- * This class is aimed at managing dictionary files for any new addition and deletion
- * and calling of clear cache for BTree and dictionary instances from LRU cache
- */
-public class ManageDictionaryAndBTree {
-
-  /**
-   * Attribute for Carbon LOGGER
-   */
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(ManageDictionaryAndBTree.class.getName());
-
-  /**
-   * This method will delete the dictionary files for the given column IDs and
-   * clear the dictionary cache
-   *
-   * @param columnSchema
-   * @param identifier
-   */
-  public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema,
-      AbsoluteTableIdentifier identifier) {
-    String metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath());
-    CarbonFile metadataDir = FileFactory.getCarbonFile(metadataDirectoryPath);
-    if (metadataDir.exists()) {
-      // sort index file is created with dictionary size appended to it. So all the files
-      // with a given column ID need to be listed
-      CarbonFile[] listFiles = metadataDir.listFiles(new CarbonFileFilter() {
-        @Override
-        public boolean accept(CarbonFile path) {
-          if (path.getName().startsWith(columnSchema.getColumnUniqueId())) {
-            return true;
-          }
-          return false;
-        }
-      });
-      for (CarbonFile file : listFiles) {
-        // try catch is inside for loop because even if one deletion fails, other files
-        // still need to be deleted
-        try {
-          FileFactory.deleteFile(file.getCanonicalPath());
-        } catch (IOException e) {
-          LOGGER.error("Failed to delete dictionary or sortIndex file for column "
-              + columnSchema.getColumnName() + "with column ID "
-              + columnSchema.getColumnUniqueId());
-        }
-      }
-    }
-    // remove dictionary cache
-    removeDictionaryColumnFromCache(identifier, columnSchema.getColumnUniqueId());
-  }
-
-  /**
-   * This mwthod will invalidate both BTree and dictionary instances from LRU cache
-   *
-   * @param carbonTable
-   */
-  public static void clearBTreeAndDictionaryLRUCache(CarbonTable carbonTable) {
-    // clear dictionary cache from LRU cache
-    List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
-    for (CarbonDimension dimension : dimensions) {
-      removeDictionaryColumnFromCache(carbonTable.getAbsoluteTableIdentifier(),
-          dimension.getColumnId());
-    }
-  }
-
-  /**
-   * This method will remove dictionary cache from driver for both reverse and forward dictionary
-   *
-   * @param carbonTableIdentifier
-   * @param columnId
-   */
-  public static void removeDictionaryColumnFromCache(AbsoluteTableIdentifier carbonTableIdentifier,
-      String columnId) {
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
-        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-            new ColumnIdentifier(columnId, null, null));
-    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
-    dictCache = CacheProvider.getInstance().createCache(CacheType.FORWARD_DICTIONARY);
-    dictCache.invalidate(dictionaryColumnUniqueIdentifier);
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionary.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionary.java
deleted file mode 100644
index c1587d6..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionary.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-/**
- * This class will be used for dictionary key and value look up
- */
-public class ReverseDictionary implements Dictionary {
-
-  /**
-   * Object which will hold the information related to this dictionary column
-   */
-  private ColumnReverseDictionaryInfo columnReverseDictionaryInfo;
-
-  /**
-   * @param columnReverseDictionaryInfo
-   */
-  public ReverseDictionary(ColumnReverseDictionaryInfo columnReverseDictionaryInfo) {
-    this.columnReverseDictionaryInfo = columnReverseDictionaryInfo;
-  }
-
-  /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value
-   * @return if found returns key else INVALID_SURROGATE_KEY
-   */
-  @Override
-  public int getSurrogateKey(String value) {
-    return columnReverseDictionaryInfo.getSurrogateKey(value);
-  }
-
-  /**
-   * This method will find and return the surrogate key for a given dictionary value
-   * Applicable scenario:
-   * 1. Incremental data load : Dictionary will not be generated for existing values. For
-   * that values have to be looked up in the existing dictionary cache.
-   * 2. Filter scenarios where from value surrogate key has to be found.
-   *
-   * @param value dictionary value as byte array
-   * @return if found returns key else INVALID_SURROGATE_KEY
-   */
-  @Override
-  public int getSurrogateKey(byte[] value) {
-    return columnReverseDictionaryInfo.getSurrogateKey(value);
-  }
-
-  /**
-   * This method will find and return the dictionary value for a given surrogate key.
-   * Applicable scenarios:
-   * 1. Query final result preparation : While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return value if found else null
-   */
-  @Override
-  public String getDictionaryValueForKey(int surrogateKey) {
-    return columnReverseDictionaryInfo.getDictionaryValueForKey(surrogateKey);
-  }
-
-  /**
-   * This method will find and return the dictionary value for a given surrogate key in bytes.
-   * Applicable scenarios:
-   * 1. Query final result preparation : While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return value if found else null
-   */
-  @Override
-  public byte[] getDictionaryValueForKeyInBytes(int surrogateKey) {
-    return columnReverseDictionaryInfo.getDictionaryValueForKeyInBytes(surrogateKey);
-  }
-
-  /**
-   * This method will find and return the sort index for a given dictionary id.
-   * Applicable scenarios:
-   * 1. Used in case of order by queries when data sorting is required
-   *
-   * @param surrogateKey a unique ID for a dictionary value
-   * @return if found returns key else 0
-   */
-  @Override
-  public int getSortedIndex(int surrogateKey) {
-    return columnReverseDictionaryInfo.getSortedIndex(surrogateKey);
-  }
-
-  /**
-   * This method will find and return the dictionary value from sorted index.
-   * Applicable scenarios:
-   * 1. Query final result preparation in case of order by queries:
-   * While convert the final result which will
-   * be surrogate key back to original dictionary values this method will be used
-   *
-   * @param sortedIndex sort index of dictionary value
-   * @return value if found else null
-   */
-  @Override
-  public String getDictionaryValueFromSortedIndex(int sortedIndex) {
-    return columnReverseDictionaryInfo.getDictionaryValueFromSortedIndex(sortedIndex);
-  }
-
-  /**
-   * The method return the dictionary chunks wrapper of a column
-   * The wrapper wraps the list<list<bye[]>> and provide the iterator to retrieve the chunks
-   * members.
-   * Applications Scenario:
-   * For preparing the column Sort info while writing the sort index file.
-   *
-   * @return
-   */
-  @Override
-  public DictionaryChunksWrapper getDictionaryChunks() {
-    return columnReverseDictionaryInfo.getDictionaryChunks();
-  }
-
-  /**
-   * This method will release the objects and set default value for primitive types
-   */
-  @Override
-  public void clear() {
-    if (null != columnReverseDictionaryInfo) {
-      columnReverseDictionaryInfo.clear();
-      columnReverseDictionaryInfo = null;
-    }
-  }
-
-  /**
-   * This method return the access count associated with the dictionary.
-   * @return
-   */
-  @Override
-  public int getAccessCount() {
-    if (null != columnReverseDictionaryInfo) {
-      return columnReverseDictionaryInfo.getAccessCount();
-    }
-    return 0;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
deleted file mode 100644
index f88da3e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.CarbonLRUCache;
-import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.ObjectSizeCalculator;
-
-import org.apache.log4j.Logger;
-
-/**
- * This class implements methods to create dictionary cache which will hold
- * dictionary chunks for look up of surrogate keys and values
- */
-public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
-    V extends Dictionary>
-    extends AbstractDictionaryCache<K, V> {
-
-  /**
-   * Attribute for Carbon LOGGER
-   */
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(ReverseDictionaryCache.class.getName());
-
-  private static final long sizeOfEmptyDictChunks =
-      ObjectSizeCalculator.estimate(new ArrayList<byte[]>(CarbonUtil.getDictionaryChunkSize()), 16);
-
-  private static final long sizeOfEmptyHashMap = ObjectSizeCalculator.estimate(new
-      ConcurrentHashMap<DictionaryByteArrayWrapper,
-          Integer>(CarbonUtil.getDictionaryChunkSize()), 16);
-
-  private static final long sizeOfHashMapNode = ObjectSizeCalculator.estimate(new
-      DictionaryByteArrayWrapper(new byte[0]), 16) +
-      ObjectSizeCalculator.estimate(0, 16);
-
-  private static final long byteArraySize = ObjectSizeCalculator.estimate(new byte[0], 16);
-
-  /**
-   * @param carbonLRUCache
-   */
-  public ReverseDictionaryCache(CarbonLRUCache carbonLRUCache) {
-    super(carbonLRUCache);
-  }
-
-  /**
-   * This method will get the value for the given key. If value does not exist
-   * for the given key, it will check and load the value.
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return dictionary
-   * @throws IOException in case memory is not sufficient to load dictionary into memory
-   */
-  @Override
-  public Dictionary get(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier)
-      throws IOException {
-    return getDictionary(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * This method will return a list of values for the given list of keys.
-   * For each key, this method will check and load the data if required.
-   *
-   * @param dictionaryColumnUniqueIdentifiers unique identifier which contains dbName,
-   *                                          tableName and columnIdentifier
-   * @return list of dictionary
-   * @throws IOException in case memory is not sufficient to load dictionary into memory
-   */
-  @Override
-  public List<Dictionary> getAll(
-      List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers)
-      throws IOException {
-    boolean exceptionOccurredInDictionaryLoading = false;
-    String exceptionMessage = "";
-    List<Dictionary> reverseDictionaryObjectList =
-        new ArrayList<Dictionary>(dictionaryColumnUniqueIdentifiers.size());
-    List<Future<Dictionary>> taskSubmitList =
-        new ArrayList<>(dictionaryColumnUniqueIdentifiers.size());
-    ExecutorService executorService = Executors.newFixedThreadPool(threadPoolSize);
-    for (final DictionaryColumnUniqueIdentifier uniqueIdent : dictionaryColumnUniqueIdentifiers) {
-      taskSubmitList.add(executorService.submit(new Callable<Dictionary>() {
-        @Override
-        public Dictionary call() throws IOException {
-          return getDictionary(uniqueIdent);
-        }
-      }));
-    }
-    try {
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.HOURS);
-    } catch (InterruptedException e) {
-      LOGGER.error("Error loading the dictionary: " + e.getMessage(), e);
-    }
-    for (int i = 0; i < taskSubmitList.size(); i++) {
-      try {
-        Dictionary columnDictionary = taskSubmitList.get(i).get();
-        reverseDictionaryObjectList.add(columnDictionary);
-      } catch (Throwable e) {
-        exceptionOccurredInDictionaryLoading = true;
-        exceptionMessage = e.getMessage();
-      }
-    }
-    if (exceptionOccurredInDictionaryLoading) {
-      clearDictionary(reverseDictionaryObjectList);
-      LOGGER.error(exceptionMessage);
-      throw new IOException(exceptionMessage);
-    }
-    return reverseDictionaryObjectList;
-  }
-
-  /**
-   * This method will return the value for the given key. It will not check and load
-   * the data for the given key
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return
-   */
-  @Override
-  public Dictionary getIfPresent(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    Dictionary reverseDictionary = null;
-    ColumnReverseDictionaryInfo columnReverseDictionaryInfo =
-        (ColumnReverseDictionaryInfo) carbonLRUCache.get(
-            getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-                CacheType.REVERSE_DICTIONARY));
-    if (null != columnReverseDictionaryInfo) {
-      reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
-      incrementDictionaryAccessCount(columnReverseDictionaryInfo);
-    }
-    return reverseDictionary;
-  }
-
-  /**
-   * This method will remove the cache for a given key
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   */
-  @Override
-  public void invalidate(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    carbonLRUCache.remove(
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-            CacheType.REVERSE_DICTIONARY));
-  }
-
-  /**
-   * This method will get the value for the given key. If value does not exist
-   * for the given key, it will check and load the value.
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return dictionary
-   * @throws IOException in case memory is not sufficient to load dictionary into memory
-   */
-  private Dictionary getDictionary(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier)
-      throws IOException {
-    Dictionary reverseDictionary = null;
-    // dictionary is only for primitive data type
-    assert (!dictionaryColumnUniqueIdentifier.getDataType().isComplexType());
-    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().getColumnId(),
-            CacheType.REVERSE_DICTIONARY), false);
-    reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
-    return reverseDictionary;
-  }
-
-  /**
-   * This method will check and create columnReverseDictionaryInfo object for the given column
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @param columnIdentifier
-   * @return
-   */
-  private ColumnReverseDictionaryInfo getColumnReverseDictionaryInfo(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, String columnIdentifier) {
-    ColumnReverseDictionaryInfo columnReverseDictionaryInfo =
-        (ColumnReverseDictionaryInfo) carbonLRUCache
-            .get(getLruCacheKey(columnIdentifier, CacheType.REVERSE_DICTIONARY));
-    if (null == columnReverseDictionaryInfo) {
-      synchronized (dictionaryColumnUniqueIdentifier) {
-        columnReverseDictionaryInfo = (ColumnReverseDictionaryInfo) carbonLRUCache
-            .get(getLruCacheKey(columnIdentifier, CacheType.REVERSE_DICTIONARY));
-        if (null == columnReverseDictionaryInfo) {
-          columnReverseDictionaryInfo = new ColumnReverseDictionaryInfo();
-        }
-      }
-    }
-    return columnReverseDictionaryInfo;
-  }
-
-  @Override
-  public void clearAccessCount(List<DictionaryColumnUniqueIdentifier> keys) {
-    for (DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier : keys) {
-      Dictionary cacheable = (Dictionary) carbonLRUCache.get(
-          getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
-              CacheType.REVERSE_DICTIONARY));
-      cacheable.clear();
-    }
-  }
-
-  @Override
-  protected long getEstimatedDictionarySize(DictionaryInfo dictionaryInfo,
-      CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier, boolean
-      readSortIndexSize) throws IOException {
-    // required size will be size total size of file - offset till file is
-    // already read
-    long requiredSize =
-        carbonDictionaryColumnMetaChunk.getEnd_offset() -
-            dictionaryInfo.getOffsetTillFileIsRead();
-
-    long numOfRecords = dictionaryInfo.getOffsetTillFileIsRead() == 0 ?
-        carbonDictionaryColumnMetaChunk.getMax_surrogate_key() :
-        carbonDictionaryColumnMetaChunk.getMax_surrogate_key()
-            - getNumRecordsInCarbonDictionaryColumnMetaChunk(
-            dictionaryColumnUniqueIdentifier,
-            dictionaryInfo.getOffsetTillFileIsRead());
-
-    if (numOfRecords > 0) {
-      long avgRecordsSize = requiredSize / numOfRecords;
-      long bytesPerRecord = (long)Math.ceil(avgRecordsSize / 8.0) * 8;
-
-      requiredSize = (bytesPerRecord + byteArraySize) * numOfRecords;
-    }
-
-    if (readSortIndexSize) {
-      // every time we are loading all the sort index files.Hence memory calculation for all
-      // the records
-      requiredSize = requiredSize + getSortIndexSize(
-          carbonDictionaryColumnMetaChunk.getMax_surrogate_key());
-    }
-
-    requiredSize = requiredSize + (sizeOfHashMapNode * numOfRecords);
-
-    return requiredSize + sizeOfEmptyDictChunks + sizeOfEmptyHashMap;
-  }
-}
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 0eed0fa..fa88027 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
@@ -275,12 +275,6 @@
 
   public static final int CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX = 10000;
 
-  @CarbonProperty
-  public static final String CARBON_SECURE_DICTIONARY_SERVER =
-      "carbon.secure.dictionary.server";
-
-  public static final String CARBON_SECURE_DICTIONARY_SERVER_DEFAULT = "true";
-
   /**
    * ENABLE_CALCULATE_DATA_INDEX_SIZE
    */
@@ -433,14 +427,9 @@
    */
   public static final String INDEX_COLUMNS = "INDEX_COLUMNS";
 
-  /**
-   * key for dictionary path
-   */
-  public static final String DICTIONARY_PATH = "dictionary_path";
   public static final String SORT_COLUMNS = "sort_columns";
   public static final String SORT_SCOPE = "sort_scope";
   public static final String RANGE_COLUMN = "range_column";
-  public static final String PARTITION_TYPE = "partition_type";
   public static final String COLUMN_PROPERTIES = "columnproperties";
   // table block size in MB
   public static final String TABLE_BLOCKSIZE = "table_blocksize";
@@ -699,17 +688,6 @@
   public static final String DICTIONARY_ONE_CHUNK_SIZE_DEFAULT = "10000";
 
   /**
-   *  Dictionary Server Worker Threads
-   */
-  @CarbonProperty
-  public static final String DICTIONARY_WORKER_THREADS = "dictionary.worker.threads";
-
-  /**
-   *  Dictionary Server Worker Threads
-   */
-  public static final String DICTIONARY_WORKER_THREADS_DEFAULT = "1";
-
-  /**
    * Size of Major Compaction in MBs
    */
   @CarbonProperty(dynamicConfigurable = true)
@@ -1003,16 +981,6 @@
 
   public static final String LOAD_GLOBAL_SORT_PARTITIONS_DEFAULT = "0";
 
-  /*
-   * carbon dictionary server port
-   */
-  @CarbonProperty
-  public static final String DICTIONARY_SERVER_PORT = "carbon.dictionary.server.port";
-
-  /**
-   * Default carbon dictionary server port
-   */
-  public static final String DICTIONARY_SERVER_PORT_DEFAULT = "2030";
   /**
    * whether to prefetch data while loading.
    */
@@ -2072,12 +2040,6 @@
   public static final String CARBON_WRITTEN_BY_APPNAME = "carbon.writtenby.app.name";
 
   /**
-   * When more global dictionary columns are there then there is issue in generating codegen to them
-   * and it slows down the query.So we limit to 100 for now
-   */
-  public static final int CARBON_ALLOW_DIRECT_FILL_DICT_COLS_LIMIT = 100;
-
-  /**
    * page size in mb. If page size exceeds this value before 32000 rows count, page will be cut.
    * And remaining rows will written in next page.
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index 8cb3f27..3a8aa3d 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -87,15 +87,6 @@
   public static final String CARBON_TABLE_LOAD_SORT_SCOPE = "carbon.table.load.sort.scope.";
 
   /**
-   * Option to enable/ disable single_pass
-   */
-  @CarbonProperty(dynamicConfigurable = true)
-  public static final String CARBON_OPTIONS_SINGLE_PASS =
-      "carbon.options.single.pass";
-
-  public static final String CARBON_OPTIONS_SINGLE_PASS_DEFAULT = "false";
-
-  /**
    * specify bad record path option
    */
   @CarbonProperty(dynamicConfigurable = true)
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
index e34dc7a..1815eb1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
@@ -23,6 +23,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.datatype.DecimalType;
@@ -113,14 +114,7 @@
           noSortDictDimSpec.add(spec);
         }
       } else if (dimension.isGlobalDictionaryEncoding()) {
-        spec = new DimensionSpec(ColumnType.GLOBAL_DICTIONARY, dimension, dictActualPosition++);
-        dimensionSpec[dimIndex++] = spec;
-        dictDimensionSpec.add(spec);
-        if (dimension.isSortColumn()) {
-          dictSortDimSpec.add(spec);
-        } else {
-          noSortDictDimSpec.add(spec);
-        }
+        throw new DeprecatedFeatureException("global dictionary");
       } else {
         spec = new DimensionSpec(ColumnType.PLAIN_VALUE, dimension, noDictActualPosition++);
         dimensionSpec[dimIndex++] = spec;
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
index eb3a1f8..776ed1a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/DefaultEncodingFactory.java
@@ -29,7 +29,6 @@
 import org.apache.carbondata.core.datastore.page.encoding.adaptive.AdaptiveIntegralCodec;
 import org.apache.carbondata.core.datastore.page.encoding.compress.DirectCompressCodec;
 import org.apache.carbondata.core.datastore.page.encoding.dimension.legacy.ComplexDimensionIndexCodec;
-import org.apache.carbondata.core.datastore.page.encoding.dimension.legacy.DictDimensionIndexCodec;
 import org.apache.carbondata.core.datastore.page.encoding.dimension.legacy.DirectDictDimensionIndexCodec;
 import org.apache.carbondata.core.datastore.page.encoding.dimension.legacy.HighCardDictDimensionIndexCodec;
 import org.apache.carbondata.core.datastore.page.statistics.PrimitivePageStatsCollector;
@@ -77,7 +76,6 @@
   private ColumnPageEncoder createEncoderForDimension(TableSpec.DimensionSpec columnSpec,
       ColumnPage inputPage) {
     switch (columnSpec.getColumnType()) {
-      case GLOBAL_DICTIONARY:
       case DIRECT_DICTIONARY:
       case PLAIN_VALUE:
         return new DirectCompressCodec(inputPage.getDataType()).createEncoder(null);
@@ -90,11 +88,6 @@
 
   private ColumnPageEncoder createEncoderForDimensionLegacy(TableSpec.DimensionSpec dimensionSpec) {
     switch (dimensionSpec.getColumnType()) {
-      case GLOBAL_DICTIONARY:
-        return new DictDimensionIndexCodec(
-            dimensionSpec.isInSortColumns(),
-            dimensionSpec.isInSortColumns() && dimensionSpec.isDoInvertedIndex())
-            .createEncoder(null);
       case DIRECT_DICTIONARY:
         return new DirectDictDimensionIndexCodec(
             dimensionSpec.isInSortColumns(),
diff --git a/core/src/main/java/org/apache/carbondata/core/devapi/BiDictionary.java b/core/src/main/java/org/apache/carbondata/core/devapi/BiDictionary.java
index 53058ed..ba72401 100644
--- a/core/src/main/java/org/apache/carbondata/core/devapi/BiDictionary.java
+++ b/core/src/main/java/org/apache/carbondata/core/devapi/BiDictionary.java
@@ -25,7 +25,7 @@
    * @param value dictionary value
    * @return dictionary key
    */
-  K getOrGenerateKey(V value) throws DictionaryGenerationException;
+  K getOrGenerateKey(V value);
 
   /**
    * Get the dictionary key corresponding to the input value, return null if value is not exist in
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java b/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
deleted file mode 100644
index fce0338..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/client/DictionaryClient.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.client;
-
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-
-public interface DictionaryClient {
-
-  void startClient(String secretKey, String address, int port, boolean encryptSecureServer);
-
-  void shutDown();
-
-  DictionaryMessage getDictionary(DictionaryMessage key);
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClient.java b/core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClient.java
deleted file mode 100644
index 21cdbee..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClient.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.client;
-
-import java.net.InetSocketAddress;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-
-import io.netty.bootstrap.Bootstrap;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioSocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import org.apache.log4j.Logger;
-
-/**
- * Dictionary client to connect to Dictionary server and generate dictionary values
- */
-public class NonSecureDictionaryClient implements DictionaryClient {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(NonSecureDictionaryClient.class.getName());
-
-  private NonSecureDictionaryClientHandler nonSecureDictionaryClientHandler =
-      new NonSecureDictionaryClientHandler();
-
-  private NioEventLoopGroup workerGroup;
-
-  /**
-   * start dictionary client
-   *
-   * @param address
-   * @param port
-   */
-  @Override
-  public void startClient(String secretKey, String address, int port,
-      boolean encryptSecureServer) {
-    LOGGER.info("Starting client on " + address + " " + port);
-    long start = System.currentTimeMillis();
-    // Create an Event with 1 thread.
-    workerGroup = new NioEventLoopGroup(1);
-    Bootstrap clientBootstrap = new Bootstrap();
-    clientBootstrap.group(workerGroup).channel(NioSocketChannel.class)
-        .handler(new ChannelInitializer<SocketChannel>() {
-          @Override
-          public void initChannel(SocketChannel ch) throws Exception {
-            ChannelPipeline pipeline = ch.pipeline();
-            // Based on length provided at header, it collects all packets
-            pipeline
-                .addLast("LengthDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 2, 0, 2));
-            pipeline.addLast("NonSecureDictionaryClientHandler", nonSecureDictionaryClientHandler);
-          }
-        });
-    clientBootstrap.connect(new InetSocketAddress(address, port));
-    LOGGER.info(
-        "Dictionary client Started, Total time spent : " + (System.currentTimeMillis() - start));
-  }
-
-  /**
-   * for client request
-   *
-   * @param key
-   * @return
-   */
-  @Override
-  public DictionaryMessage getDictionary(
-      DictionaryMessage key) {
-    return nonSecureDictionaryClientHandler.getDictionary(key);
-  }
-
-  /**
-   * shutdown dictionary client
-   */
-  @Override
-  public void shutDown() {
-    workerGroup.shutdownGracefully();
-    try {
-      workerGroup.terminationFuture().sync();
-    } catch (InterruptedException e) {
-      LOGGER.error(e.getMessage(), e);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClientHandler.java b/core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClientHandler.java
deleted file mode 100644
index 1af65bd..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/client/NonSecureDictionaryClientHandler.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.client;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.log4j.Logger;
-
-/**
- * Client handler to get data.
- */
-public class NonSecureDictionaryClientHandler extends ChannelInboundHandlerAdapter {
-
-  private static final Logger LOGGER =
-          LogServiceFactory.getLogService(NonSecureDictionaryClientHandler.class.getName());
-
-  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
-
-  private ChannelHandlerContext ctx;
-
-  private DictionaryChannelFutureListener channelFutureListener;
-
-  @Override
-  public void channelActive(ChannelHandlerContext ctx) throws Exception {
-    this.ctx = ctx;
-    channelFutureListener = new DictionaryChannelFutureListener(ctx);
-    LOGGER.info("Connected client " + ctx);
-    super.channelActive(ctx);
-  }
-
-  @Override
-  public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-    try {
-      ByteBuf data = (ByteBuf) msg;
-      DictionaryMessage key = new DictionaryMessage();
-      key.readSkipLength(data);
-      data.release();
-      responseMsgQueue.add(key);
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      throw e;
-    }
-  }
-
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
-    LOGGER.error("exceptionCaught", cause);
-    ctx.close();
-  }
-
-  /**
-   * client send request to server
-   *
-   * @param key DictionaryMessage
-   * @return DictionaryMessage
-   */
-  public DictionaryMessage getDictionary(DictionaryMessage key) {
-    DictionaryMessage dictionaryMessage;
-    try {
-      ByteBuf buffer = ctx.alloc().buffer();
-      key.writeData(buffer);
-      ctx.writeAndFlush(buffer).addListener(channelFutureListener);
-    } catch (Exception e) {
-      LOGGER.error("Error while send request to server ", e);
-      ctx.close();
-    }
-    try {
-      dictionaryMessage = responseMsgQueue.poll(100, TimeUnit.SECONDS);
-      if (dictionaryMessage == null) {
-        StringBuilder message = new StringBuilder();
-        message.append("DictionaryMessage { ColumnName: ")
-            .append(key.getColumnName())
-            .append(", DictionaryValue: ")
-            .append(key.getDictionaryValue())
-            .append(", type: ")
-            .append(key.getType());
-        throw new RuntimeException("Request timed out for key : " + message);
-      }
-      return dictionaryMessage;
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      throw new RuntimeException(e);
-    }
-  }
-
-  private static class DictionaryChannelFutureListener implements ChannelFutureListener {
-
-    private ChannelHandlerContext ctx;
-
-    DictionaryChannelFutureListener(ChannelHandlerContext ctx) {
-      this.ctx = ctx;
-    }
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if (!future.isSuccess()) {
-        LOGGER.error("Error while sending request to Dictionary Server", future.cause());
-        ctx.close();
-      }
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java
deleted file mode 100644
index e33da1c..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.io.IOException;
-
-/**
- * Interface to write dictionary to file system
- */
-public interface DictionaryWriter {
-
-  void writeDictionaryData() throws IOException;
-
-}
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
deleted file mode 100644
index a3600e7..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-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.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-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.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator;
-
-import org.apache.log4j.Logger;
-
-/**
- * This generator does not maintain the whole cache of dictionary. It just maintains the cache only
- * for the loading session, so what ever the dictionary values it generates in the loading session
- * it keeps in cache.
- */
-public class IncrementalColumnDictionaryGenerator implements BiDictionary<Integer, String>,
-        DictionaryGenerator<Integer, String>, DictionaryWriter {
-
-  private static final Logger LOGGER =
-          LogServiceFactory.getLogService(IncrementalColumnDictionaryGenerator.class.getName());
-
-  private final Object lock = new Object();
-
-  private Map<String, Integer> incrementalCache = new ConcurrentHashMap<>();
-
-  private Map<Integer, String> reverseIncrementalCache = new ConcurrentHashMap<>();
-
-  private int currentDictionarySize;
-
-  private int maxValue;
-
-  private CarbonColumn dimension;
-
-  private CarbonTable carbonTable;
-
-  public IncrementalColumnDictionaryGenerator(CarbonColumn dimension, int maxValue,
-      CarbonTable carbonTable) {
-    this.carbonTable = carbonTable;
-    this.maxValue = maxValue;
-    this.currentDictionarySize = maxValue;
-    this.dimension = dimension;
-  }
-
-  @Override
-  public Integer getOrGenerateKey(String value) throws DictionaryGenerationException {
-    Integer dict = getKey(value);
-    if (dict == null) {
-      dict = generateKey(value);
-    }
-    return dict;
-  }
-
-  @Override
-  public Integer getKey(String value) {
-    return incrementalCache.get(value);
-  }
-
-  @Override
-  public String getValue(Integer key) {
-    return reverseIncrementalCache.get(key);
-  }
-
-  @Override
-  public int size() {
-    synchronized (lock) {
-      return currentDictionarySize;
-    }
-  }
-
-  @Override
-  public Integer generateKey(String value) throws DictionaryGenerationException {
-    synchronized (lock) {
-      Integer dict = incrementalCache.get(value);
-      if (dict == null) {
-        dict = ++currentDictionarySize;
-        incrementalCache.put(value, dict);
-        reverseIncrementalCache.put(dict, value);
-      }
-      return dict;
-    }
-  }
-
-  @Override
-  public void writeDictionaryData() throws IOException {
-    // initialize params
-    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());
-    Boolean isDictExists = CarbonUtil.isFileExistsForGivenColumn(identifier);
-    Dictionary dictionary = null;
-    long t1 = System.currentTimeMillis();
-    if (isDictExists) {
-      Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache = CacheProvider.getInstance()
-              .createCache(CacheType.REVERSE_DICTIONARY);
-      dictionary = dictCache.get(identifier);
-    }
-    long dictCacheTime = System.currentTimeMillis() - t1;
-    long t2 = System.currentTimeMillis();
-    // write dictionary
-    CarbonDictionaryWriter dictionaryWriter = null;
-    dictionaryWriter = dictionaryService.getDictionaryWriter(identifier);
-    List<String> distinctValues = writeDictionary(dictionaryWriter, isDictExists);
-    long dictWriteTime = System.currentTimeMillis() - t2;
-    long t3 = System.currentTimeMillis();
-    // write sort index
-    if (distinctValues.size() > 0) {
-      writeSortIndex(distinctValues, dictionary,
-              dictionaryService, absoluteTableIdentifier, columnIdentifier);
-    }
-    long sortIndexWriteTime = System.currentTimeMillis() - t3;
-    // update Meta Data
-    updateMetaData(dictionaryWriter);
-    LOGGER.info("\n columnName: " + dimension.getColName() +
-            "\n columnId: " + dimension.getColumnId() +
-            "\n new distinct values count: " + distinctValues.size() +
-            "\n create dictionary cache: " + dictCacheTime +
-            "\n sort list, distinct and write: " + dictWriteTime +
-            "\n write sort info: " + sortIndexWriteTime);
-
-    if (isDictExists) {
-      CarbonUtil.clearDictionaryCache(dictionary);
-    }
-  }
-
-  /**
-   * write dictionary to file
-   *
-   * @param dictionaryWriter
-   * @param isDictExists
-   * @return
-   * @throws IOException
-   */
-  private List<String> writeDictionary(CarbonDictionaryWriter dictionaryWriter,
-                                       Boolean isDictExists) throws IOException {
-    List<String> distinctValues = new ArrayList<>();
-    try {
-      if (!isDictExists) {
-        dictionaryWriter.write(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-        distinctValues.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-      }
-      // write value to dictionary file
-      if (reverseIncrementalCache.size() > 0) {
-        synchronized (lock) {
-          // collect incremental dictionary
-          for (int index = maxValue + 1; index <= currentDictionarySize; index++) {
-            String value = reverseIncrementalCache.get(index);
-            String parsedValue = DataTypeUtil.normalizeColumnValueForItsDataType(value, dimension);
-            if (null != parsedValue) {
-              dictionaryWriter.write(parsedValue);
-              distinctValues.add(parsedValue);
-            }
-          }
-          // clear incremental dictionary to avoid write to file again
-          reverseIncrementalCache.clear();
-          incrementalCache.clear();
-          currentDictionarySize = maxValue;
-        }
-      }
-    } finally {
-      if (null != dictionaryWriter) {
-        dictionaryWriter.close();
-      }
-    }
-
-    return distinctValues;
-  }
-
-  /**
-   * write dictionary sort index to file
-   *
-   * @param distinctValues
-   * @param dictionary
-   * @param dictionaryService
-   * @param absoluteTableIdentifier
-   * @param columnIdentifier
-   * @throws IOException
-   */
-  private void writeSortIndex(List<String> distinctValues,
-                              Dictionary dictionary,
-                              DictionaryService dictionaryService,
-                              AbsoluteTableIdentifier absoluteTableIdentifier,
-                              ColumnIdentifier columnIdentifier) throws IOException {
-    CarbonDictionarySortIndexWriter carbonDictionarySortIndexWriter = null;
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    try {
-      CarbonDictionarySortInfoPreparator preparator = new CarbonDictionarySortInfoPreparator();
-      CarbonDictionarySortInfo dictionarySortInfo =
-              preparator.getDictionarySortInfo(distinctValues, dictionary,
-                      dimension.getDataType());
-      carbonDictionarySortIndexWriter = dictionaryService
-          .getDictionarySortIndexWriter(dictionaryColumnUniqueIdentifier);
-      carbonDictionarySortIndexWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
-      carbonDictionarySortIndexWriter
-              .writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
-    } finally {
-      if (null != carbonDictionarySortIndexWriter) {
-        carbonDictionarySortIndexWriter.close();
-      }
-    }
-  }
-
-  /**
-   * update dictionary metadata
-   *
-   * @param dictionaryWriter
-   * @throws IOException
-   */
-  private void updateMetaData(CarbonDictionaryWriter dictionaryWriter) throws IOException {
-    if (null != dictionaryWriter) {
-      dictionaryWriter.commit();
-    }
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
deleted file mode 100644
index 5a08552..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-/**
- * This is the dictionary generator for all tables. It generates dictionary
- * based on @{@link DictionaryMessage}.
- */
-public class ServerDictionaryGenerator implements DictionaryGenerator<Integer, DictionaryMessage> {
-
-  /**
-   * the map of tableName to TableDictionaryGenerator
-   */
-  private Map<String, TableDictionaryGenerator> tableMap = new ConcurrentHashMap<>();
-
-  @Override
-  public Integer generateKey(DictionaryMessage value)
-      throws DictionaryGenerationException {
-    initializeGeneratorForColumn(value);
-    TableDictionaryGenerator generator = tableMap.get(value.getTableUniqueId());
-    return generator.generateKey(value);
-  }
-
-  public void initializeGeneratorForTable(CarbonTable carbonTable) {
-    // initialize TableDictionaryGenerator first
-    String tableId = carbonTable.getCarbonTableIdentifier().getTableId();
-    if (tableMap.get(tableId) == null) {
-      synchronized (tableMap) {
-        if (tableMap.get(tableId) == null) {
-          tableMap.put(tableId,
-              new TableDictionaryGenerator(carbonTable));
-        }
-      }
-    }
-  }
-
-  public void initializeGeneratorForColumn(DictionaryMessage key) {
-    tableMap.get(key.getTableUniqueId()).updateGenerator(key);
-  }
-
-  public Integer size(DictionaryMessage key) {
-    initializeGeneratorForColumn(key);
-    TableDictionaryGenerator generator = tableMap.get(key.getTableUniqueId());
-    return generator.size(key);
-  }
-
-  public void writeTableDictionaryData(String tableUniqueId) throws Exception {
-    TableDictionaryGenerator generator = tableMap.get(tableUniqueId);
-    if (generator != null) {
-      generator.writeDictionaryData();
-    }
-    // Remove dictionary generator after writing
-    tableMap.remove(tableUniqueId);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
deleted file mode 100644
index b212291..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-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.CarbonProperties;
-
-import org.apache.log4j.Logger;
-
-/**
- * Dictionary generation for table.
- */
-public class TableDictionaryGenerator
-    implements DictionaryGenerator<Integer, DictionaryMessage>, DictionaryWriter {
-
-  private static final Logger LOGGER =
-          LogServiceFactory.getLogService(TableDictionaryGenerator.class.getName());
-
-  private CarbonTable carbonTable;
-  /**
-   * the map of columnName to dictionaryGenerator
-   */
-  private Map<String, DictionaryGenerator<Integer, String>> columnMap = new ConcurrentHashMap<>();
-
-  public TableDictionaryGenerator(CarbonTable carbonTable) {
-    this.carbonTable = carbonTable;
-  }
-
-  @Override
-  public Integer generateKey(DictionaryMessage value)
-      throws DictionaryGenerationException {
-    CarbonColumn dimension = carbonTable.getPrimitiveDimensionByName(value.getColumnName());
-
-    if (null == dimension) {
-      throw new DictionaryGenerationException("Dictionary Generation Failed");
-    }
-    DictionaryGenerator<Integer, String> generator =
-            columnMap.get(dimension.getColumnId());
-    return generator.generateKey(value.getData());
-  }
-
-  public Integer size(DictionaryMessage key) {
-    CarbonColumn dimension = carbonTable.getPrimitiveDimensionByName(key.getColumnName());
-
-    if (null == dimension) {
-      return 0;
-    }
-    DictionaryGenerator<Integer, String> generator =
-            columnMap.get(dimension.getColumnId());
-    return ((BiDictionary) generator).size();
-  }
-
-  @Override
-  public void writeDictionaryData() {
-    int numOfCores = CarbonProperties.getInstance().getNumberOfLoadingCores();
-    long start = System.currentTimeMillis();
-    ExecutorService executorService = Executors.newFixedThreadPool(numOfCores);
-    for (final DictionaryGenerator generator : columnMap.values()) {
-      executorService.execute(new WriteDictionaryDataRunnable(generator));
-    }
-
-    try {
-      executorService.shutdown();
-      executorService.awaitTermination(1, TimeUnit.HOURS);
-    } catch (InterruptedException e) {
-      LOGGER.error("Error loading the dictionary: " + e.getMessage(), e);
-    }
-    LOGGER.info("Total time taken to write dictionary file is: " +
-            (System.currentTimeMillis() - start));
-  }
-
-  public void updateGenerator(DictionaryMessage key) {
-    CarbonColumn dimension = carbonTable
-        .getPrimitiveDimensionByName(key.getColumnName());
-    if (null != dimension && null == columnMap.get(dimension.getColumnId())) {
-      synchronized (columnMap) {
-        if (null == columnMap.get(dimension.getColumnId())) {
-          columnMap.put(dimension.getColumnId(),
-              new IncrementalColumnDictionaryGenerator(dimension, 1, carbonTable));
-        }
-      }
-    }
-  }
-
-  private static class WriteDictionaryDataRunnable implements Runnable {
-    private final DictionaryGenerator generator;
-
-    public WriteDictionaryDataRunnable(DictionaryGenerator generator) {
-      this.generator = generator;
-    }
-
-    @Override
-    public void run() {
-      try {
-        ((DictionaryWriter)generator).writeDictionaryData();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java
deleted file mode 100644
index 1b9c4ae..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator.key;
-
-import java.nio.charset.Charset;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-import io.netty.buffer.ByteBuf;
-
-/**
- * Dictionary key to generate dictionary
- */
-public class DictionaryMessage {
-
-  /**
-   * tableUniqueId
-   */
-  private String tableUniqueId;
-
-  /**
-   * columnName
-   */
-  private String columnName;
-
-  /**
-   * message data
-   */
-  private String data;
-
-  /**
-   * Dictionary Value
-   */
-  private int dictionaryValue = CarbonCommonConstants.INVALID_SURROGATE_KEY;
-
-  /**
-   * message type
-   */
-  private DictionaryMessageType type;
-
-  public void readSkipLength(ByteBuf byteBuf) {
-
-    byte[] tableBytes = new byte[byteBuf.readInt()];
-    byteBuf.readBytes(tableBytes);
-    tableUniqueId = new String(tableBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-
-    byte[] colBytes = new byte[byteBuf.readInt()];
-    byteBuf.readBytes(colBytes);
-    columnName = new String(colBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-
-    byte typeByte = byteBuf.readByte();
-    type = getKeyType(typeByte);
-
-    byte dataType = byteBuf.readByte();
-    if (dataType == 0) {
-      dictionaryValue = byteBuf.readInt();
-    } else {
-      byte[] dataBytes = new byte[byteBuf.readInt()];
-      byteBuf.readBytes(dataBytes);
-      data = new String(dataBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    }
-  }
-
-  public void readFullLength(ByteBuf byteBuf) {
-
-    byteBuf.readShort();
-    byte[] tableIdBytes = new byte[byteBuf.readInt()];
-    byteBuf.readBytes(tableIdBytes);
-    tableUniqueId =
-        new String(tableIdBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-
-    byte[] colBytes = new byte[byteBuf.readInt()];
-    byteBuf.readBytes(colBytes);
-    columnName = new String(colBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-
-    byte typeByte = byteBuf.readByte();
-    type = getKeyType(typeByte);
-
-    byte dataType = byteBuf.readByte();
-    if (dataType == 0) {
-      dictionaryValue = byteBuf.readInt();
-    } else {
-      byte[] dataBytes = new byte[byteBuf.readInt()];
-      byteBuf.readBytes(dataBytes);
-      data = new String(dataBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    }
-  }
-
-  public void writeData(ByteBuf byteBuf) {
-    int startIndex = byteBuf.writerIndex();
-    // Just reserve the bytes to add length of header at last.
-    byteBuf.writeShort(Short.MAX_VALUE);
-
-    byte[] tableIdBytes =
-        tableUniqueId.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    byteBuf.writeInt(tableIdBytes.length);
-    byteBuf.writeBytes(tableIdBytes);
-
-    byte[] colBytes = columnName.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    byteBuf.writeInt(colBytes.length);
-    byteBuf.writeBytes(colBytes);
-
-    byteBuf.writeByte(type.getType());
-
-    if (dictionaryValue > 0) {
-      byteBuf.writeByte(0);
-      byteBuf.writeInt(dictionaryValue);
-    } else {
-      byteBuf.writeByte(1);
-      byte[] dataBytes = data.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-      byteBuf.writeInt(dataBytes.length);
-      byteBuf.writeBytes(dataBytes);
-    }
-    int endIndex = byteBuf.writerIndex();
-    // Add the length of message at the starting.it is required while decoding as in TCP protocol
-    // it not guarantee that we receive all data in one packet, so we need to wait to receive all
-    // packets before proceeding to process the message.Based on the length it waits.
-    byteBuf.setShort(startIndex, endIndex - startIndex - 2);
-  }
-
-  private DictionaryMessageType getKeyType(byte type) {
-    switch (type) {
-      case 2:
-        return DictionaryMessageType.SIZE;
-      case 3:
-        return DictionaryMessageType.WRITE_TABLE_DICTIONARY;
-      default:
-        return DictionaryMessageType.DICT_GENERATION;
-    }
-  }
-
-  public String getColumnName() {
-    return columnName;
-  }
-
-  public String getData() {
-    return data;
-  }
-
-  public void setData(String data) {
-    this.data = data;
-  }
-
-  public DictionaryMessageType getType() {
-    return type;
-  }
-
-  public void setType(DictionaryMessageType type) {
-    this.type = type;
-  }
-
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
-
-  public int getDictionaryValue() {
-    return dictionaryValue;
-  }
-
-  public void setDictionaryValue(int dictionaryValue) {
-    this.dictionaryValue = dictionaryValue;
-  }
-
-  public String getTableUniqueId() {
-    return tableUniqueId;
-  }
-
-  public void setTableUniqueId(String tableUniqueId) {
-    this.tableUniqueId = tableUniqueId;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java
deleted file mode 100644
index 3281fd3..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator.key;
-
-/**
- * Dictionary key types.
- */
-public enum DictionaryMessageType {
-
-  DICT_GENERATION((byte) 1),
-  SIZE((byte) 2),
-  WRITE_TABLE_DICTIONARY((byte) 3);
-
-  final byte type;
-
-  DictionaryMessageType(byte type) {
-    this.type = type;
-  }
-
-  public byte getType() {
-    return type;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java
deleted file mode 100644
index f814281..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.server;
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-public interface DictionaryServer {
-
-  void startServer();
-
-  void bindToPort();
-
-  void shutdown()throws Exception;
-
-  String getHost();
-
-  int getPort();
-
-  String getSecretKey();
-
-  boolean isEncryptSecureServer();
-
-  void writeTableDictionary(String uniqueTableName) throws Exception;
-
-  void initializeDictionaryGenerator(CarbonTable carbonTable) throws Exception;
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
deleted file mode 100644
index 0b8c800..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.server;
-
-import java.net.InetSocketAddress;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-import org.apache.carbondata.core.dictionary.service.AbstractDictionaryServer;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.ChannelInitializer;
-import io.netty.channel.ChannelOption;
-import io.netty.channel.ChannelPipeline;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
-import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import org.apache.log4j.Logger;
-
-/**
- * Dictionary Server to generate dictionary keys.
- */
-public class NonSecureDictionaryServer extends AbstractDictionaryServer
-    implements DictionaryServer {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(NonSecureDictionaryServer.class.getName());
-
-  private NonSecureDictionaryServerHandler nonSecureDictionaryServerHandler;
-
-  private EventLoopGroup boss;
-  private EventLoopGroup worker;
-  private int port;
-  private String host;
-  private static Object lock = new Object();
-  private static NonSecureDictionaryServer INSTANCE = null;
-
-  private NonSecureDictionaryServer(int port) {
-    this.port = port;
-    startServer();
-  }
-
-  public static synchronized DictionaryServer getInstance(int port, CarbonTable carbonTable)
-      throws Exception {
-    if (INSTANCE == null) {
-      INSTANCE = new NonSecureDictionaryServer(port);
-    }
-    INSTANCE.initializeDictionaryGenerator(carbonTable);
-    return INSTANCE;
-  }
-
-  /**
-   * start dictionary server
-   *
-   */
-  @Override
-  public void startServer() {
-    LOGGER.info("Starting Dictionary Server in Non Secure Mode");
-    nonSecureDictionaryServerHandler = new NonSecureDictionaryServerHandler();
-    String workerThreads = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
-            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
-    boss = new NioEventLoopGroup(1);
-    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
-    // Configure the server.
-    bindToPort();
-  }
-
-  /**
-   * Binds dictionary server to an available port.
-   *
-   */
-  @Override
-  public void bindToPort() {
-    long start = System.currentTimeMillis();
-    // Configure the server.
-    int i = 0;
-    while (i < 10) {
-      int newPort = port + i;
-      try {
-        ServerBootstrap bootstrap = new ServerBootstrap();
-        bootstrap.group(boss, worker);
-        bootstrap.channel(NioServerSocketChannel.class);
-        bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
-          @Override
-          public void initChannel(SocketChannel ch) throws Exception {
-            ChannelPipeline pipeline = ch.pipeline();
-            pipeline
-                .addLast("LengthDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 2, 0, 2));
-            pipeline.addLast("NonSecureDictionaryServerHandler", nonSecureDictionaryServerHandler);
-          }
-        });
-        bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true);
-        String hostToBind = findLocalIpAddress(LOGGER);
-        //iteratively listening to newports
-        InetSocketAddress address = hostToBind == null ?
-            new InetSocketAddress(newPort) :
-            new InetSocketAddress(hostToBind, newPort);
-        bootstrap.bind(address).sync();
-        LOGGER.info("Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
-            + " Listening on port " + newPort);
-        this.port = newPort;
-        this.host = hostToBind;
-        break;
-      } catch (Exception e) {
-        LOGGER.error("Dictionary Server Failed to bind to port:" + newPort, e);
-        if (i == 9) {
-          throw new RuntimeException("Dictionary Server Could not bind to any port");
-        }
-      }
-      i++;
-    }
-  }
-
-  /**
-   * @return Port on which the NonSecureDictionaryServer has started.
-   */
-  @Override
-  public int getPort() {
-    return port;
-  }
-
-  @Override
-  public String getSecretKey() {
-    return null;
-  }
-
-  @Override
-  public boolean isEncryptSecureServer() {
-    return false;
-  }
-
-  @Override
-  public String getHost() {
-    return host;
-  }
-
-  /**
-   * shutdown dictionary server
-   *
-   * @throws Exception
-   */
-  @Override
-  public void shutdown() throws Exception {
-    LOGGER.info("Shutting down dictionary server");
-    worker.shutdownGracefully();
-    boss.shutdownGracefully();
-  }
-
-  /**
-   * Write Dictionary for one table.
-   *
-   * @throws Exception
-   */
-
-  @Override
-  public void writeTableDictionary(String uniqueTableName) throws Exception {
-    DictionaryMessage key = new DictionaryMessage();
-    key.setTableUniqueId(uniqueTableName);
-    key.setType(DictionaryMessageType.WRITE_TABLE_DICTIONARY);
-    nonSecureDictionaryServerHandler.processMessage(key);
-  }
-
-  public void initializeDictionaryGenerator(CarbonTable carbonTable) throws Exception {
-    nonSecureDictionaryServerHandler.initializeTable(carbonTable);
-  }
-
-}
\ No newline at end of file
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServerHandler.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServerHandler.java
deleted file mode 100644
index 13c682b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServerHandler.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.server;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.dictionary.generator.ServerDictionaryGenerator;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundHandlerAdapter;
-import org.apache.log4j.Logger;
-
-/**
- * Handler for Dictionary server.
- */
-@ChannelHandler.Sharable public class NonSecureDictionaryServerHandler
-    extends ChannelInboundHandlerAdapter {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(NonSecureDictionaryServerHandler.class.getName());
-
-  /**
-   * dictionary generator
-   */
-  private ServerDictionaryGenerator generatorForServer = new ServerDictionaryGenerator();
-
-  /**
-   * channel registered
-   *
-   * @param ctx
-   * @throws Exception
-   */
-  public void channelActive(ChannelHandlerContext ctx) throws Exception {
-    LOGGER.info("Connected " + ctx);
-    super.channelActive(ctx);
-  }
-
-  @Override
-  public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
-    try {
-      ByteBuf data = (ByteBuf) msg;
-      DictionaryMessage key = new DictionaryMessage();
-      key.readSkipLength(data);
-      data.release();
-      int outPut = processMessage(key);
-      key.setDictionaryValue(outPut);
-      // Send back the response
-      ByteBuf buffer = ctx.alloc().buffer();
-      key.writeData(buffer);
-      ctx.writeAndFlush(buffer);
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      throw e;
-    }
-  }
-
-  /**
-   * handle exceptions
-   *
-   * @param ctx
-   * @param cause
-   */
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
-    LOGGER.error("exceptionCaught", cause);
-    ctx.close();
-  }
-
-  /**
-   * process message by message type
-   *
-   * @param key
-   * @return
-   * @throws Exception
-   */
-  public int processMessage(DictionaryMessage key) throws Exception {
-    switch (key.getType()) {
-      case DICT_GENERATION:
-        generatorForServer.initializeGeneratorForColumn(key);
-        return generatorForServer.generateKey(key);
-      case SIZE:
-        generatorForServer.initializeGeneratorForColumn(key);
-        return generatorForServer.size(key);
-      case WRITE_TABLE_DICTIONARY:
-        generatorForServer
-            .writeTableDictionaryData(key.getTableUniqueId());
-        return 0;
-      default:
-        return -1;
-    }
-  }
-
-  void initializeTable(CarbonTable carbonTable) {
-    generatorForServer.initializeGeneratorForTable(carbonTable);
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/service/AbstractDictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/service/AbstractDictionaryServer.java
deleted file mode 100644
index f548646..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/service/AbstractDictionaryServer.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.service;
-
-import java.net.Inet4Address;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.List;
-
-import org.apache.commons.lang3.SystemUtils;
-import org.apache.log4j.Logger;
-
-public abstract class AbstractDictionaryServer {
-
-  public String findLocalIpAddress(Logger LOGGER) {
-    try {
-      String defaultIpOverride = System.getenv("SPARK_LOCAL_IP");
-      if (defaultIpOverride != null) {
-        return defaultIpOverride;
-      } else {
-        InetAddress address = InetAddress.getLocalHost();
-        if (address.isLoopbackAddress()) {
-          // Address resolves to something like 127.0.1.1, which happens on Debian; try to find
-          // a better address using the local network interfaces
-          // getNetworkInterfaces returns ifs in reverse order compared to ifconfig output order
-          // on unix-like system. On windows, it returns in index order.
-          // It's more proper to pick ip address following system output order.
-          Enumeration<NetworkInterface> activeNetworkIFs = NetworkInterface.getNetworkInterfaces();
-          List<NetworkInterface> reOrderedNetworkIFs = new ArrayList<NetworkInterface>();
-          while (activeNetworkIFs.hasMoreElements()) {
-            reOrderedNetworkIFs.add(activeNetworkIFs.nextElement());
-          }
-
-          if (!SystemUtils.IS_OS_WINDOWS) {
-            Collections.reverse(reOrderedNetworkIFs);
-          }
-
-          for (NetworkInterface ni : reOrderedNetworkIFs) {
-            Enumeration<InetAddress> inetAddresses = ni.getInetAddresses();
-            while (inetAddresses.hasMoreElements()) {
-              InetAddress addr = inetAddresses.nextElement();
-              if (!addr.isLinkLocalAddress() && !addr.isLoopbackAddress()
-                  && addr instanceof Inet4Address) {
-                // We've found an address that looks reasonable!
-                LOGGER.warn("Your hostname, " + InetAddress.getLocalHost().getHostName()
-                    + " resolves to a loopback address: " + address.getHostAddress() + "; using "
-                    + addr.getHostAddress() + " instead (on interface " + ni.getName() + ")");
-                LOGGER.warn("Set SPARK_LOCAL_IP if you need to bind to another address");
-                return addr.getHostAddress();
-              }
-            }
-            LOGGER.warn("Your hostname, " + InetAddress.getLocalHost().getHostName()
-                + " resolves to a loopback address: " + address.getHostAddress()
-                + ", but we couldn't find any external IP address!");
-            LOGGER.warn("Set SPARK_LOCAL_IP if you need to bind to another address");
-          }
-        }
-        return address.getHostAddress();
-      }
-    } catch (UnknownHostException e) {
-      LOGGER.error("do not get local host address:" + e.getMessage(), e);
-      throw new RuntimeException(e);
-    } catch (SocketException e) {
-      LOGGER.error("do not get net work interface:" + e.getMessage(), e);
-      throw new RuntimeException(e);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryOnePassService.java b/core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryOnePassService.java
deleted file mode 100644
index 48c6295..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryOnePassService.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.service;
-
-public class DictionaryOnePassService {
-  private static DictionaryServiceProvider dictionaryServiceProvider = null;
-
-  public static void setDictionaryServiceProvider(DictionaryServiceProvider dictionaryServiceProv) {
-    dictionaryServiceProvider = dictionaryServiceProv;
-  }
-
-  public static synchronized DictionaryServiceProvider getDictionaryProvider() {
-    return dictionaryServiceProvider;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/service/NonSecureDictionaryServiceProvider.java b/core/src/main/java/org/apache/carbondata/core/dictionary/service/NonSecureDictionaryServiceProvider.java
deleted file mode 100644
index 0063753..0000000
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/service/NonSecureDictionaryServiceProvider.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.service;
-
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.client.NonSecureDictionaryClient;
-
-public class NonSecureDictionaryServiceProvider implements DictionaryServiceProvider {
-  private int port = 0;
-
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  public NonSecureDictionaryServiceProvider(int port) {
-    this.port = port;
-  }
-
-  //  @Override public DictionaryServer getDictionaryServer() {
-  //    return NonSecureDictionaryServer.getInstance(port);
-  //  }
-  @Override
-  public DictionaryClient getDictionaryClient() {
-    return new NonSecureDictionaryClient();
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
index c865603..6a4405e 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
@@ -206,7 +206,7 @@
    * @throws IOException
    */
   private SegmentProperties initSegmentProperties(BlockletDataMapModel blockletDataMapInfo,
-      DataFileFooter fileFooter) throws IOException {
+      DataFileFooter fileFooter) {
     List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
     int[] columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
     segmentPropertiesWrapper = SegmentPropertiesAndSchemaHolder.getInstance()
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
index 7544afb..5d99872 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
@@ -38,12 +38,6 @@
   private String tablePath;
 
   /**
-   * dictionary path of the table
-   */
-  private String dictionaryPath;
-
-
-  /**
    * carbon table identifier which will have table name and table database
    * name
    */
@@ -158,11 +152,4 @@
     return carbonTableIdentifier.toString();
   }
 
-  public String getDictionaryPath() {
-    return dictionaryPath;
-  }
-
-  public void setDictionaryPath(String dictionaryPath) {
-    this.dictionaryPath = dictionaryPath;
-  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 86ce9ba..c35d3e0 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -702,11 +702,7 @@
    * @return absolute table identifier
    */
   public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        tableInfo.getOrCreateAbsoluteTableIdentifier();
-    absoluteTableIdentifier.setDictionaryPath(
-        tableInfo.getFactTable().getTableProperties().get(CarbonCommonConstants.DICTIONARY_PATH));
-    return absoluteTableIdentifier;
+    return tableInfo.getOrCreateAbsoluteTableIdentifier();
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
index 522e46b..262db8f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
@@ -72,7 +72,7 @@
     this.ordinal = ordinal;
     this.schemaOrdinal = schemaOrdinal;
     this.columnIdentifier =
-     new ColumnIdentifier(getColumnId(), getColumnProperties(), getDataType());
+        new ColumnIdentifier(getColumnId(), getColumnProperties(), getDataType());
   }
 
   /**
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryServiceProvider.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnUniqueIdGenerator.java
similarity index 75%
rename from core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryServiceProvider.java
rename to core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnUniqueIdGenerator.java
index 5d7d813..0573111 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/service/DictionaryServiceProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnUniqueIdGenerator.java
@@ -15,13 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.dictionary.service;
+package org.apache.carbondata.core.metadata.schema.table.column;
 
-import java.io.Serializable;
+import java.util.UUID;
 
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+/**
+ * It returns unique id given column
+ */
+public class ColumnUniqueIdGenerator {
 
-public interface DictionaryServiceProvider extends Serializable {
+  public static String generateUniqueId() {
+    return UUID.randomUUID().toString();
+  }
 
-  DictionaryClient getDictionaryClient();
 }
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
deleted file mode 100644
index d3afc54..0000000
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.reader;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
-
-import org.apache.thrift.TBase;
-
-/**
- * This class perform the functionality of reading the dictionary metadata file
- */
-public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetadataReader {
-
-  /**
-   * column identifier
-   */
-  protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  /**
-   * dictionary metadata file path
-   */
-  protected String columnDictionaryMetadataFilePath;
-
-  /**
-   * dictionary metadata thrift file reader
-   */
-  private ThriftReader dictionaryMetadataFileReader;
-
-  /**
-   * Constructor
-   *
-   * @param dictionaryColumnUniqueIdentifier column unique identifier
-   */
-  public CarbonDictionaryMetadataReaderImpl(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-    initFileLocation();
-  }
-
-  /**
-   * This method will be used to read complete metadata file.
-   * Applicable scenarios:
-   * 1. Query execution. Whenever a query is executed then to read the dictionary file
-   * and define the query scope first dictionary metadata has to be read first.
-   * 2. If dictionary file is read using start and end offset then using this meta list
-   * we can count the total number of dictionary chunks present between the 2 offsets
-   *
-   * @return list of all dictionary meta chunks which contains information for each segment
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public List<CarbonDictionaryColumnMetaChunk> read() throws IOException {
-    List<CarbonDictionaryColumnMetaChunk> dictionaryMetaChunks =
-        new ArrayList<CarbonDictionaryColumnMetaChunk>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    CarbonDictionaryColumnMetaChunk columnMetaChunk = null;
-    ColumnDictionaryChunkMeta dictionaryChunkMeta = null;
-    // open dictionary meta thrift reader
-    openThriftReader();
-    // read till dictionary chunk count
-    while (dictionaryMetadataFileReader.hasNext()) {
-      // get the thrift object for dictionary chunk
-      dictionaryChunkMeta = (ColumnDictionaryChunkMeta) dictionaryMetadataFileReader.read();
-      // create a new instance of chunk meta wrapper using thrift object
-      columnMetaChunk = getNewInstanceOfCarbonDictionaryColumnMetaChunk(dictionaryChunkMeta);
-      dictionaryMetaChunks.add(columnMetaChunk);
-    }
-    return dictionaryMetaChunks;
-  }
-
-  /**
-   * This method will be used to read only the last entry of dictionary meta chunk.
-   * Applicable scenarios :
-   * 1. Global dictionary generation for incremental load. In this case only the
-   * last dictionary chunk meta entry has to be read to calculate min, max surrogate
-   * key and start and end offset for the new dictionary chunk.
-   * 2. Truncate operation. While writing dictionary file in case of incremental load
-   * dictionary file needs to be validated for any inconsistency. Here end offset of last
-   * dictionary chunk meta is validated with file size.
-   *
-   * @return last segment entry for dictionary chunk
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public CarbonDictionaryColumnMetaChunk readLastEntryOfDictionaryMetaChunk()
-      throws IOException {
-    ColumnDictionaryChunkMeta dictionaryChunkMeta = null;
-    // open dictionary meta thrift reader
-    openThriftReader();
-    // at the completion of while loop we will get the last dictionary chunk entry
-    while (dictionaryMetadataFileReader.hasNext()) {
-      // get the thrift object for dictionary chunk
-      dictionaryChunkMeta = (ColumnDictionaryChunkMeta) dictionaryMetadataFileReader.read();
-    }
-    if (null == dictionaryChunkMeta) {
-      throw new IOException("Last dictionary chunk does not exist");
-    }
-    // create a new instance of chunk meta wrapper using thrift object
-    return getNewInstanceOfCarbonDictionaryColumnMetaChunk(dictionaryChunkMeta);
-  }
-
-  @Override
-  public CarbonDictionaryColumnMetaChunk readEntryOfDictionaryMetaChunk(long end_Offset)
-          throws IOException {
-    ColumnDictionaryChunkMeta dictionaryChunkMeta = null;
-    // open dictionary meta thrift reader
-    openThriftReader();
-    // at the completion of while loop we will get the last dictionary chunk entry
-    while (dictionaryMetadataFileReader.hasNext()) {
-      // get the thrift object for dictionary chunk
-      dictionaryChunkMeta = (ColumnDictionaryChunkMeta) dictionaryMetadataFileReader.read();
-      if (dictionaryChunkMeta.end_offset >= end_Offset) {
-        break;
-      }
-    }
-    if (null == dictionaryChunkMeta) {
-      throw new IOException("Matching dictionary chunk does not exist");
-    }
-    // create a new instance of chunk meta wrapper using thrift object
-    return getNewInstanceOfCarbonDictionaryColumnMetaChunk(dictionaryChunkMeta);
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated
-   * with it. If the stream is already closed then invoking this
-   * method has no effect.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-    if (null != dictionaryMetadataFileReader) {
-      dictionaryMetadataFileReader.close();
-      dictionaryMetadataFileReader = null;
-    }
-  }
-
-  /**
-   * This method will form the path for dictionary metadata file for a given column
-   */
-  protected void initFileLocation() {
-    this.columnDictionaryMetadataFilePath =
-        dictionaryColumnUniqueIdentifier.getDictionaryMetaFilePath();
-  }
-
-  /**
-   * This method will open the dictionary file stream for reading
-   *
-   * @throws IOException thrift reader open method throws IOException
-   */
-  private void openThriftReader() throws IOException {
-    // initialise dictionary file reader which will return dictionary thrift object
-    // dictionary thrift object contains a list of byte buffer
-    if (null == dictionaryMetadataFileReader) {
-      dictionaryMetadataFileReader =
-          new ThriftReader(this.columnDictionaryMetadataFilePath, new ThriftReader.TBaseCreator() {
-            @Override
-            public TBase create() {
-              return new ColumnDictionaryChunkMeta();
-            }
-          });
-      // Open it
-      dictionaryMetadataFileReader.open();
-    }
-
-  }
-
-  /**
-   * Given a thrift object thie method will create a new wrapper class object
-   * for dictionary chunk
-   *
-   * @param dictionaryChunkMeta reference for chunk meta thrift object
-   * @return wrapper object of dictionary chunk meta
-   */
-  private CarbonDictionaryColumnMetaChunk getNewInstanceOfCarbonDictionaryColumnMetaChunk(
-      ColumnDictionaryChunkMeta dictionaryChunkMeta) {
-    return new CarbonDictionaryColumnMetaChunk(dictionaryChunkMeta.getMin_surrogate_key(),
-        dictionaryChunkMeta.getMax_surrogate_key(), dictionaryChunkMeta.getStart_offset(),
-        dictionaryChunkMeta.getEnd_offset(), dictionaryChunkMeta.getChunk_count());
-  }
-}
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
deleted file mode 100644
index f559655..0000000
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ /dev/null
@@ -1,311 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.reader;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.dictionary.ColumnDictionaryChunkIterator;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.format.ColumnDictionaryChunk;
-
-import org.apache.thrift.TBase;
-
-/**
- * This class performs the functionality of reading a carbon dictionary file.
- * It implements various overloaded method for read functionality.
- */
-public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
-
-  /**
-   * column name
-   */
-  protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  /**
-   * dictionary file path
-   */
-  protected String columnDictionaryFilePath;
-
-  /**
-   * dictionary thrift file reader
-   */
-  private ThriftReader dictionaryFileReader;
-
-  /**
-   * Constructor
-   *
-   * @param dictionaryColumnUniqueIdentifier      column unique identifier
-   */
-  public CarbonDictionaryReaderImpl(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-    initFileLocation();
-  }
-
-  /**
-   * This method should be used when complete dictionary data needs to be read.
-   * Applicable scenarios :
-   * 1. Global dictionary generation in case of incremental load
-   * 2. Reading dictionary file on first time query
-   * 3. Loading a dictionary column in memory based on query requirement.
-   * This is a case where carbon column cache feature is enabled in which a
-   * column dictionary is read if it is present in the query.
-   *
-   * @return list of byte array. Each byte array is unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public List<byte[]> read() throws IOException {
-    return read(0L);
-  }
-
-  /**
-   * This method should be used when data has to be read from a given offset.
-   * Applicable scenarios :
-   * 1. Incremental data load. If column dictionary is already loaded in memory
-   * and incremental load is done, then for the new query only new dictionary data
-   * has to be read form memory.
-   *
-   * @param startOffset start offset of dictionary file
-   * @return list of byte array. Each byte array is unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public List<byte[]> read(long startOffset) throws IOException {
-    List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
-        readDictionaryMetadataFile();
-    // get the last entry for carbon dictionary meta chunk
-    CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk =
-        carbonDictionaryColumnMetaChunks.get(carbonDictionaryColumnMetaChunks.size() - 1);
-    // end offset till where the dictionary file has to be read
-    long endOffset = carbonDictionaryColumnMetaChunk.getEnd_offset();
-    List<ColumnDictionaryChunk> columnDictionaryChunks =
-        read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
-    return getDictionaryList(columnDictionaryChunks);
-  }
-
-  /**
-   * This method will be used to read data between given start and end offset.
-   * Applicable scenarios:
-   * 1. Truncate operation. If there is any inconsistency while writing the dictionary file
-   * then we can give the start and end offset till where the data has to be retained.
-   *
-   * @param startOffset start offset of dictionary file
-   * @param endOffset   end offset of dictionary file
-   * @return iterator over byte array. Each byte array is unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public Iterator<byte[]> read(long startOffset, long endOffset) throws IOException {
-    List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
-        readDictionaryMetadataFile();
-    List<ColumnDictionaryChunk> columnDictionaryChunks =
-        read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
-    return (Iterator<byte[]>) new ColumnDictionaryChunkIterator(columnDictionaryChunks);
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated
-   * with it. If the stream is already closed then invoking this
-   * method has no effect.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-    if (null != dictionaryFileReader) {
-      dictionaryFileReader.close();
-      dictionaryFileReader = null;
-    }
-  }
-
-  /**
-   * @param carbonDictionaryColumnMetaChunks dictionary meta chunk list
-   * @param startOffset                      start offset for dictionary data file
-   * @param endOffset                        end offset till where data has
-   *                                         to be read from dictionary data file
-   * @return list of byte column dictionary values
-   * @throws IOException readDictionary file method throws IO exception
-   */
-  private List<ColumnDictionaryChunk> read(
-      List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks, long startOffset,
-      long endOffset) throws IOException {
-    // calculate the number of chunks to be read from dictionary file from start offset
-    int dictionaryChunkCountsToBeRead =
-        calculateTotalDictionaryChunkCountsToBeRead(carbonDictionaryColumnMetaChunks, startOffset,
-            endOffset);
-    // open dictionary file thrift reader
-    openThriftReader();
-    // read the required number of chunks from dictionary file
-    return readDictionaryFile(startOffset, dictionaryChunkCountsToBeRead);
-  }
-
-  /**
-   * This method will put all the dictionary chunks into one list and return that list
-   *
-   * @param columnDictionaryChunks
-   * @return
-   */
-  private List<byte[]> getDictionaryList(List<ColumnDictionaryChunk> columnDictionaryChunks) {
-    int dictionaryListSize = 0;
-    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
-      dictionaryListSize = dictionaryListSize + dictionaryChunk.getValues().size();
-    }
-    // convert byte buffer list to byte array list of dictionary values
-    List<byte[]> dictionaryValues = new ArrayList<byte[]>(dictionaryListSize);
-    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
-      convertAndFillByteBufferListToByteArrayList(dictionaryValues, dictionaryChunk.getValues());
-    }
-    return dictionaryValues;
-  }
-
-  /**
-   * This method will convert and fill list of byte buffer to list of byte array
-   *
-   * @param dictionaryValues          list of byte array. Each byte array is
-   *                                  unique dictionary value
-   * @param dictionaryValueBufferList dictionary thrift object which is a list of byte buffer.
-   *                                  Each dictionary value is a wrapped in byte buffer before
-   *                                  writing to file
-   */
-  private void convertAndFillByteBufferListToByteArrayList(List<byte[]> dictionaryValues,
-      List<ByteBuffer> dictionaryValueBufferList) {
-    for (ByteBuffer buffer : dictionaryValueBufferList) {
-      int length = buffer.limit();
-      byte[] value = new byte[length];
-      buffer.get(value, 0, value.length);
-      dictionaryValues.add(value);
-    }
-  }
-
-  /**
-   * This method will form the path for dictionary file for a given column
-   */
-  protected void initFileLocation() {
-    this.columnDictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
-  }
-
-  /**
-   * This method will read the dictionary file and return the list of dictionary thrift object
-   *
-   * @param dictionaryStartOffset        start offset for dictionary file
-   * @param dictionaryChunkCountToBeRead number of dictionary chunks to be read
-   * @return list of dictionary chunks
-   * @throws IOException setReadOffset method throws I/O exception
-   */
-  private List<ColumnDictionaryChunk> readDictionaryFile(long dictionaryStartOffset,
-      int dictionaryChunkCountToBeRead) throws IOException {
-    List<ColumnDictionaryChunk> dictionaryChunks =
-        new ArrayList<ColumnDictionaryChunk>(dictionaryChunkCountToBeRead);
-    // skip the number of bytes if a start offset is given
-    dictionaryFileReader.setReadOffset(dictionaryStartOffset);
-    // read till dictionary chunk count
-    while (dictionaryFileReader.hasNext()
-        && dictionaryChunks.size() != dictionaryChunkCountToBeRead) {
-      dictionaryChunks.add((ColumnDictionaryChunk) dictionaryFileReader.read());
-    }
-    return dictionaryChunks;
-  }
-
-  /**
-   * This method will read the dictionary metadata file for a given column
-   * and calculate the number of chunks to be read from the dictionary file.
-   * It will do a strict validation for start and end offset as if the offsets are not
-   * exactly matching, because data is written in thrift format, the thrift object
-   * will not be retrieved properly
-   *
-   * @param dictionaryChunkMetaList    list of dictionary chunk metadata
-   * @param dictionaryChunkStartOffset start offset for a dictionary chunk
-   * @param dictionaryChunkEndOffset   end offset for a dictionary chunk
-   * @return
-   */
-  private int calculateTotalDictionaryChunkCountsToBeRead(
-      List<CarbonDictionaryColumnMetaChunk> dictionaryChunkMetaList,
-      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset) {
-    boolean chunkWithStartOffsetFound = false;
-    int dictionaryChunkCount = 0;
-    for (CarbonDictionaryColumnMetaChunk metaChunk : dictionaryChunkMetaList) {
-      // find the column meta chunk whose start offset value matches
-      // with the given dictionary start offset
-      if (!chunkWithStartOffsetFound && dictionaryChunkStartOffset == metaChunk.getStart_offset()) {
-        chunkWithStartOffsetFound = true;
-      }
-      // start offset is found then keep adding the chunk count to be read
-      if (chunkWithStartOffsetFound) {
-        dictionaryChunkCount = dictionaryChunkCount + metaChunk.getChunk_count();
-      }
-      // when end offset is reached then break the loop
-      if (dictionaryChunkEndOffset == metaChunk.getEnd_offset()) {
-        break;
-      }
-    }
-    return dictionaryChunkCount;
-  }
-
-  /**
-   * This method will read dictionary metadata file and return the dictionary meta chunks
-   *
-   * @return list of dictionary metadata chunks
-   * @throws IOException read and close method throws IO exception
-   */
-  private List<CarbonDictionaryColumnMetaChunk> readDictionaryMetadataFile() throws IOException {
-    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
-    List<CarbonDictionaryColumnMetaChunk> dictionaryMetaChunkList = null;
-    // read metadata file
-    try {
-      dictionaryMetaChunkList = columnMetadataReaderImpl.read();
-    } finally {
-      // close the metadata reader
-      columnMetadataReaderImpl.close();
-    }
-    return dictionaryMetaChunkList;
-  }
-
-  /**
-   * @return
-   */
-  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
-    return new CarbonDictionaryMetadataReaderImpl(this.dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * This method will open the dictionary file stream for reading
-   *
-   * @throws IOException thrift reader open method throws IOException
-   */
-  private void openThriftReader() throws IOException {
-    if (null == dictionaryFileReader) {
-      // initialise dictionary file reader which will return dictionary thrift object
-      // dictionary thrift object contains a list of byte buffer
-      dictionaryFileReader =
-          new ThriftReader(this.columnDictionaryFilePath, new ThriftReader.TBaseCreator() {
-            @Override
-            public TBase create() {
-              return new ColumnDictionaryChunk();
-            }
-          });
-      // Open dictionary file reader
-      dictionaryFileReader.open();
-    }
-
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReader.java b/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReader.java
deleted file mode 100644
index 891bf0f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReader.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.reader.sortindex;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * Interface for reading the dictionary sort index and sort index inverted
- */
-public interface CarbonDictionarySortIndexReader extends Closeable {
-
-  /**
-   * method for reading the carbon dictionary sort index data
-   * from columns sortIndex file.
-   *
-   * @return The method return's the list of dictionary sort Index and sort Index reverse
-   * @throws IOException In case any I/O error occurs
-   */
-  List<Integer> readSortIndex() throws IOException;
-
-  /**
-   * method for reading the carbon dictionary inverted sort index data
-   * from columns sortIndex file.
-   *
-   * @return The method return's the list of dictionary inverted sort Index
-   * @throws IOException In case any I/O error occurs
-   */
-  List<Integer> readInvertedSortIndex() throws IOException;
-}
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
deleted file mode 100644
index fe57013..0000000
--- a/core/src/main/java/org/apache/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.reader.sortindex;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-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.reader.ThriftReader;
-import org.apache.carbondata.format.ColumnSortInfo;
-
-import org.apache.log4j.Logger;
-import org.apache.thrift.TBase;
-
-/**
- * Implementation for reading the dictionary sort index and inverted sort index .
- */
-public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySortIndexReader {
-
-  /**
-   * column name
-   */
-  protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  /**
-   * the path of the dictionary Sort Index file
-   */
-  protected String sortIndexFilePath;
-
-  /**
-   * Column sort info thrift instance.
-   */
-  ColumnSortInfo columnSortInfo = null;
-
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(CarbonDictionarySortIndexReaderImpl.class.getName());
-
-  /**
-   * dictionary sortIndex file Reader
-   */
-  private ThriftReader dictionarySortIndexThriftReader;
-
-  /**
-   * @param dictionaryColumnUniqueIdentifier column name
-   */
-  public CarbonDictionarySortIndexReaderImpl(
-      final DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-  }
-
-  /**
-   * method for reading the carbon dictionary sort index data
-   * from columns sortIndex file.
-   *
-   * @return The method return's the list of dictionary sort Index and sort Index reverse
-   * In case of no member for column empty list will be return
-   * @throws IOException In case any I/O error occurs
-   */
-  @Override
-  public List<Integer> readSortIndex() throws IOException {
-    if (null == columnSortInfo) {
-      readColumnSortInfo();
-    }
-    return columnSortInfo.getSort_index();
-  }
-
-  /**
-   * method for reading the carbon dictionary sort index data
-   * from columns sortIndex file.
-   * In case of no member empty list will be return
-   *
-   * @throws IOException In case any I/O error occurs
-   */
-  private void readColumnSortInfo() throws IOException {
-    init();
-    try {
-      columnSortInfo = (ColumnSortInfo) dictionarySortIndexThriftReader.read();
-    } catch (IOException ie) {
-      LOGGER.error("problem while reading the column sort info.", ie);
-      throw new IOException("problem while reading the column sort info.", ie);
-    } finally {
-      if (null != dictionarySortIndexThriftReader) {
-        dictionarySortIndexThriftReader.close();
-      }
-    }
-  }
-
-  /**
-   * method for reading the carbon dictionary inverted sort index data
-   * from columns sortIndex file.
-   *
-   * @return The method return's the list of dictionary inverted sort Index
-   * @throws IOException In case any I/O error occurs
-   */
-  @Override
-  public List<Integer> readInvertedSortIndex() throws IOException {
-    if (null == columnSortInfo) {
-      readColumnSortInfo();
-    }
-    return columnSortInfo.getSort_index_inverted();
-  }
-
-  /**
-   * The method initializes the dictionary Sort Index file path
-   * and initialize and opens the thrift reader for dictionary sortIndex file.
-   *
-   * @throws IOException if any I/O errors occurs
-   */
-  private void init() throws IOException {
-    initPath();
-    openThriftReader();
-  }
-
-  protected void initPath() {
-    try {
-      CarbonDictionaryColumnMetaChunk chunkMetaObjectForLastSegmentEntry =
-          getChunkMetaObjectForLastSegmentEntry();
-      long dictOffset = chunkMetaObjectForLastSegmentEntry.getEnd_offset();
-      this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath(dictOffset);
-      if (!FileFactory.isFileExist(this.sortIndexFilePath)) {
-        this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath();
-      }
-    } catch (IOException e) {
-      this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath();
-    }
-
-  }
-
-  /**
-   * This method will read the dictionary chunk metadata thrift object for last entry
-   *
-   * @return last entry of dictionary meta chunk
-   * @throws IOException if an I/O error occurs
-   */
-  private CarbonDictionaryColumnMetaChunk getChunkMetaObjectForLastSegmentEntry()
-      throws IOException {
-    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
-    try {
-      // read the last segment entry for dictionary metadata
-      return columnMetadataReaderImpl.readLastEntryOfDictionaryMetaChunk();
-    } finally {
-      // Close metadata reader
-      columnMetadataReaderImpl.close();
-    }
-  }
-
-  /**
-   * @return
-   */
-  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
-    return new CarbonDictionaryMetadataReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * This method will open the dictionary sort index file stream for reading
-   *
-   * @throws IOException in case any I/O errors occurs
-   */
-  private void openThriftReader() throws IOException {
-    this.dictionarySortIndexThriftReader =
-        new ThriftReader(this.sortIndexFilePath, new ThriftReader.TBaseCreator() {
-          @Override
-          public TBase create() {
-            return new ColumnSortInfo();
-          }
-        });
-    dictionarySortIndexThriftReader.open();
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated
-   * with it. If the stream is already closed then invoking this
-   * method has no effect.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-    if (null != dictionarySortIndexThriftReader) {
-      dictionarySortIndexThriftReader.close();
-    }
-  }
-}
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 22711ca..5dfda83 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,11 +385,9 @@
           dictionarySourceAbsoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
           columnIdentifier = dimension.getColumnIdentifier();
         }
-        String dictionaryPath = carbonTable.getTableInfo().getFactTable().getTableProperties()
-            .get(CarbonCommonConstants.DICTIONARY_PATH);
         dictionaryColumnUniqueIdentifiers.add(
             new DictionaryColumnUniqueIdentifier(dictionarySourceAbsoluteTableIdentifier,
-                columnIdentifier, dimension.getDataType(), dictionaryPath));
+                columnIdentifier, dimension.getDataType()));
       }
     }
     return dictionaryColumnUniqueIdentifiers;
@@ -623,9 +621,6 @@
         boolean isDirectDictionary = CarbonUtil
             .hasEncoding(dimension.getListOfChildDimensions().get(i).getEncoder(),
                 Encoding.DIRECT_DICTIONARY);
-        boolean isDictionary = CarbonUtil
-            .hasEncoding(dimension.getListOfChildDimensions().get(i).getEncoder(),
-                Encoding.DICTIONARY);
 
         parentQueryType.addChildren(
             new PrimitiveQueryType(dimension.getListOfChildDimensions().get(i).getColName(),
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 679ee43..f1786c5 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
@@ -64,7 +64,6 @@
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
@@ -717,11 +716,10 @@
    * @param evaluateResultList
    * @param isIncludeFilter
    * @return
-   * @throws QueryExecutionException
    */
   public static ColumnFilterInfo getFilterValues(AbsoluteTableIdentifier tableIdentifier,
       ColumnExpression columnExpression, List<String> evaluateResultList, boolean isIncludeFilter)
-      throws QueryExecutionException, FilterUnsupportedException, IOException {
+      throws IOException {
     Dictionary forwardDictionary = null;
     ColumnFilterInfo filterInfo = null;
     List<Integer> surrogates =
@@ -789,7 +787,7 @@
   }
 
   private static ColumnFilterInfo getDimColumnFilterInfoAfterApplyingCBO(
-      Dictionary forwardDictionary, ColumnFilterInfo filterInfo) throws FilterUnsupportedException {
+      Dictionary forwardDictionary, ColumnFilterInfo filterInfo) {
     List<Integer> excludeMemberSurrogates =
         prepareExcludeFilterMembers(forwardDictionary, filterInfo.getFilterList());
     filterInfo.setExcludeFilterList(excludeMemberSurrogates);
@@ -838,8 +836,7 @@
   }
 
   private static List<Integer> prepareExcludeFilterMembers(
-      Dictionary forwardDictionary, List<Integer> includeSurrogates)
-      throws FilterUnsupportedException {
+      Dictionary forwardDictionary, List<Integer> includeSurrogates) {
     DictionaryChunksWrapper dictionaryWrapper;
     RoaringBitmap bitMapOfSurrogates = RoaringBitmap.bitmapOf(
         ArrayUtils.toPrimitive(includeSurrogates.toArray(new Integer[includeSurrogates.size()])));
@@ -1444,12 +1441,12 @@
    * This API will get the max value of surrogate key which will be used for
    * determining the end key of particular btree.
    *
-   * @param dimCarinality
+   * @param dimCardinality
    */
-  private static long getMaxValue(CarbonDimension carbonDimension, int[] dimCarinality) {
+  private static long getMaxValue(CarbonDimension carbonDimension, int[] dimCardinality) {
     // Get data from all the available slices of the table
-    if (null != dimCarinality) {
-      return dimCarinality[carbonDimension.getKeyOrdinal()];
+    if (null != dimCardinality) {
+      return dimCardinality[carbonDimension.getKeyOrdinal()];
     }
     return -1;
   }
@@ -1462,12 +1459,7 @@
   public static Dictionary getForwardDictionaryCache(
       AbsoluteTableIdentifier dictionarySourceAbsoluteTableIdentifier,
       CarbonDimension carbonDimension) throws IOException {
-    String dictionaryPath = null;
     ColumnIdentifier columnIdentifier = carbonDimension.getColumnIdentifier();
-    String dicPath = dictionarySourceAbsoluteTableIdentifier.getDictionaryPath();
-    if (null != dicPath && !dicPath.trim().isEmpty()) {
-      dictionaryPath = dicPath;
-    }
     if (null != carbonDimension.getColumnSchema().getParentColumnTableRelations()
         && carbonDimension.getColumnSchema().getParentColumnTableRelations().size() == 1) {
       dictionarySourceAbsoluteTableIdentifier =
@@ -1478,7 +1470,7 @@
     }
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(dictionarySourceAbsoluteTableIdentifier,
-            columnIdentifier, carbonDimension.getDataType(), dictionaryPath);
+            columnIdentifier, carbonDimension.getDataType());
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache =
         cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
index c3b069b..c35e096 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
@@ -22,7 +22,6 @@
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.core.scan.expression.exception.FilterIllegalMemberException;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.ColumnFilterInfo;
@@ -55,26 +54,22 @@
       } catch (FilterIllegalMemberException e) {
         throw new FilterUnsupportedException(e);
       }
-      try {
 
-        resolvedFilterObject = FilterUtil
-            .getFilterValues(metadata.getTableIdentifier(), metadata.getColumnExpression(),
-                evaluateResultListFinal, metadata.isIncludeFilter());
-        if (!metadata.isIncludeFilter() && null != resolvedFilterObject) {
-          // Adding default surrogate key of null member inorder to not display the same while
-          // displaying the report as per hive compatibility.
-          // first check of surrogate key for null value is already added then
-          // no need to add again otherwise result will be wrong in case of exclude filter
-          // this is because two times it will flip the same bit
-          if (!resolvedFilterObject.getExcludeFilterList()
-              .contains(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY)) {
-            resolvedFilterObject.getExcludeFilterList()
-                .add(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY);
-          }
-          Collections.sort(resolvedFilterObject.getExcludeFilterList());
+      resolvedFilterObject = FilterUtil
+          .getFilterValues(metadata.getTableIdentifier(), metadata.getColumnExpression(),
+              evaluateResultListFinal, metadata.isIncludeFilter());
+      if (!metadata.isIncludeFilter() && null != resolvedFilterObject) {
+        // Adding default surrogate key of null member inorder to not display the same while
+        // displaying the report as per hive compatibility.
+        // first check of surrogate key for null value is already added then
+        // no need to add again otherwise result will be wrong in case of exclude filter
+        // this is because two times it will flip the same bit
+        if (!resolvedFilterObject.getExcludeFilterList()
+            .contains(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY)) {
+          resolvedFilterObject.getExcludeFilterList()
+              .add(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY);
         }
-      } catch (QueryExecutionException e) {
-        throw new FilterUnsupportedException(e);
+        Collections.sort(resolvedFilterObject.getExcludeFilterList());
       }
       resolveDimension.setFilterValues(resolvedFilterObject);
     }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonDictionary.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonDictionary.java
index e036507..8075e3e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonDictionary.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonDictionary.java
@@ -31,5 +31,4 @@
 
   byte[][] getAllDictionaryValues();
 
-  byte[] getAllDictionaryValuesInSingleArray();
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonDictionaryImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonDictionaryImpl.java
index 37a462e..e7391b5 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonDictionaryImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/impl/CarbonDictionaryImpl.java
@@ -23,12 +23,6 @@
 
   private byte[][] dictionary;
 
-  private byte[] singleArrayDictValues;
-
-  private int[] dictLens;
-
-  private int[] dictOffsets;
-
   private int actualSize;
 
   private boolean isDictUsed;
@@ -68,35 +62,4 @@
     return dictionary;
   }
 
-  @Override
-  public byte[] getAllDictionaryValuesInSingleArray() {
-    if (singleArrayDictValues == null) {
-      dictLens = new int[dictionary.length];
-      dictOffsets = new int[dictionary.length];
-      int size = 0;
-      for (int i = 0; i < dictionary.length; i++) {
-        if (dictionary[i] != null) {
-          dictOffsets[i] = size;
-          size += dictionary[i].length;
-          dictLens[i] = dictionary[i].length;
-        }
-      }
-      singleArrayDictValues = new byte[size];
-      for (int i = 0; i < dictionary.length; i++) {
-        if (dictionary[i] != null) {
-          System.arraycopy(dictionary[i], 0, singleArrayDictValues, dictOffsets[i], dictLens[i]);
-        }
-      }
-      dictionary = null;
-    }
-    return singleArrayDictValues;
-  }
-
-  public int[] getDictLens() {
-    return dictLens;
-  }
-
-  public int[] getDictOffsets() {
-    return dictOffsets;
-  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
deleted file mode 100644
index 6d86bb0..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.service;
-
-import org.apache.carbondata.core.service.impl.DictionaryFactory;
-
-/**
- * Interface to get services
- */
-public class CarbonCommonFactory {
-
-  /**
-   * @return dictionary service
-   */
-  public static DictionaryService getDictionaryService() {
-    return DictionaryFactory.getInstance();
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java b/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java
deleted file mode 100644
index 068e552..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/DictionaryService.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.service;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
-import org.apache.carbondata.core.reader.CarbonDictionaryReader;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-
-/**
- * Dictionary service to get writer and reader
- */
-public interface DictionaryService {
-
-  /**
-   * get dictionary writer
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  CarbonDictionaryWriter getDictionaryWriter(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-
-  /**
-   * get dictionary sort index writer
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-
-  /**
-   * get dictionary metadata reader
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  CarbonDictionaryMetadataReader getDictionaryMetadataReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-
-  /**
-   * get dictionary reader
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  CarbonDictionaryReader getDictionaryReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-
-  /**
-   * get dictionary sort index reader
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  CarbonDictionarySortIndexReader getDictionarySortIndexReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java b/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java
deleted file mode 100644
index ba55b75..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/impl/DictionaryFactory.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.service.impl;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
-import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
-import org.apache.carbondata.core.reader.CarbonDictionaryReader;
-import org.apache.carbondata.core.reader.CarbonDictionaryReaderImpl;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
-import org.apache.carbondata.core.service.DictionaryService;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-import org.apache.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 dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  @Override
-  public CarbonDictionaryWriter getDictionaryWriter(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    return new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * get dictionary sort index writer
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  @Override
-  public CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    return new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * get dictionary metadata reader
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  @Override
-  public CarbonDictionaryMetadataReader getDictionaryMetadataReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    return new CarbonDictionaryMetadataReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * get dictionary reader
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  @Override
-  public CarbonDictionaryReader getDictionaryReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    return new CarbonDictionaryReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * get dictionary sort index reader
-   *
-   * @param dictionaryColumnUniqueIdentifier
-   * @return
-   */
-  @Override
-  public CarbonDictionarySortIndexReader getDictionarySortIndexReader(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    return new CarbonDictionarySortIndexReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  public static DictionaryService getInstance() {
-    return dictService;
-  }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonLoadStatisticsImpl.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonLoadStatisticsImpl.java
index b1d9332..ed06c73 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonLoadStatisticsImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonLoadStatisticsImpl.java
@@ -49,7 +49,6 @@
   private long loadCsvfilesToDfStartTime = 0;
   private long loadCsvfilesToDfCostTime = 0;
   private long dicShuffleAndWriteFileTotalStartTime = 0;
-  private long dicShuffleAndWriteFileTotalCostTime = 0;
 
   //LRU cache load one time
   private double lruCacheLoadTime = 0;
@@ -95,19 +94,14 @@
 
   //Record the time
   public void recordDicShuffleAndWriteTime() {
-    Long dicShuffleAndWriteTimePoint = System.currentTimeMillis();
+    long dicShuffleAndWriteTimePoint = System.currentTimeMillis();
     if (0 == dicShuffleAndWriteFileTotalStartTime) {
       dicShuffleAndWriteFileTotalStartTime = dicShuffleAndWriteTimePoint;
     }
-    if (dicShuffleAndWriteTimePoint - dicShuffleAndWriteFileTotalStartTime >
-            dicShuffleAndWriteFileTotalCostTime) {
-      dicShuffleAndWriteFileTotalCostTime =
-          dicShuffleAndWriteTimePoint - dicShuffleAndWriteFileTotalStartTime;
-    }
   }
 
   public void recordLoadCsvfilesToDfTime() {
-    Long loadCsvfilesToDfTimePoint = System.currentTimeMillis();
+    long loadCsvfilesToDfTimePoint = System.currentTimeMillis();
     if (0 == loadCsvfilesToDfStartTime) {
       loadCsvfilesToDfStartTime = loadCsvfilesToDfTimePoint;
     }
@@ -234,11 +228,6 @@
     this.totalRecords = totalRecords;
   }
 
-  //Get the time
-  private double getDicShuffleAndWriteFileTotalTime() {
-    return dicShuffleAndWriteFileTotalCostTime / 1000.0;
-  }
-
   private double getLoadCsvfilesToDfTime() {
     return loadCsvfilesToDfCostTime / 1000.0;
   }
@@ -282,10 +271,6 @@
     return (int)(totalRecords / totalTime);
   }
 
-  private int getGenDicSpeed() {
-    return (int)(totalRecords / getLoadCsvfilesToDfTime() + getDicShuffleAndWriteFileTotalTime());
-  }
-
   private int getReadCSVSpeed(String partitionID) {
     return (int)(totalRecords / getCsvInputStepTime(partitionID));
   }
@@ -303,7 +288,7 @@
   }
 
   private double getTotalTime(String partitionID) {
-    this.totalTime = getLoadCsvfilesToDfTime() + getDicShuffleAndWriteFileTotalTime() +
+    this.totalTime = getLoadCsvfilesToDfTime() +
         getLruCacheLoadTime() + getDictionaryValuesTotalTime(partitionID) +
         getDictionaryValue2MdkAdd2FileTime(partitionID);
     return totalTime;
@@ -314,35 +299,32 @@
     double loadCsvfilesToDfTime = getLoadCsvfilesToDfTime();
     LOGGER.info("STAGE 1 ->Load csv to DataFrame and generate" +
             " block distinct values: " + loadCsvfilesToDfTime + "(s)");
-    double dicShuffleAndWriteFileTotalTime = getDicShuffleAndWriteFileTotalTime();
-    LOGGER.info("STAGE 2 ->Global dict shuffle and write dict file: " +
-            dicShuffleAndWriteFileTotalTime + "(s)");
   }
 
   private void printLruCacheLoadTimeInfo() {
-    LOGGER.info("STAGE 3 ->LRU cache load: " + getLruCacheLoadTime() + "(s)");
+    LOGGER.info("STAGE 2 ->LRU cache load: " + getLruCacheLoadTime() + "(s)");
   }
 
   private void printDictionaryValuesGenStatisticsInfo(String partitionID) {
     double dictionaryValuesTotalTime = getDictionaryValuesTotalTime(partitionID);
-    LOGGER.info("STAGE 4 ->Total cost of gen dictionary values, sort and write to temp files: "
+    LOGGER.info("STAGE 3 ->Total cost of gen dictionary values, sort and write to temp files: "
             + dictionaryValuesTotalTime + "(s)");
     double csvInputStepTime = getCsvInputStepTime(partitionID);
     double generatingDictionaryValuesTime = getGeneratingDictionaryValuesTime(partitionID);
-    LOGGER.info("STAGE 4.1 ->  |_read csv file: " + csvInputStepTime + "(s)");
-    LOGGER.info("STAGE 4.2 ->  |_transform to surrogate key: "
+    LOGGER.info("STAGE 3.1 ->  |_read csv file: " + csvInputStepTime + "(s)");
+    LOGGER.info("STAGE 3.2 ->  |_transform to surrogate key: "
             + generatingDictionaryValuesTime + "(s)");
   }
 
   private void printSortRowsStepStatisticsInfo(String partitionID) {
     double sortRowsStepTotalTime = getSortRowsStepTotalTime(partitionID);
-    LOGGER.info("STAGE 4.3 ->  |_sort rows and write to temp file: "
+    LOGGER.info("STAGE 3.3 ->  |_sort rows and write to temp file: "
             + sortRowsStepTotalTime + "(s)");
   }
 
   private void printGenMdkStatisticsInfo(String partitionID) {
     double dictionaryValue2MdkAdd2FileTime = getDictionaryValue2MdkAdd2FileTime(partitionID);
-    LOGGER.info("STAGE 5 ->Transform to MDK, compress and write fact files: "
+    LOGGER.info("STAGE 4 ->Transform to MDK, compress and write fact files: "
             + dictionaryValue2MdkAdd2FileTime + "(s)");
   }
 
@@ -369,7 +351,6 @@
     LOGGER.info("Total Num of Records Processed: " + getTotalRecords());
     LOGGER.info("Total Time Cost: " + getTotalTime(partitionID) + "(s)");
     LOGGER.info("Total Load Speed: " + getLoadSpeed() + "records/s");
-    LOGGER.info("Generate Dictionaries Speed: " + getGenDicSpeed() + "records/s");
     LOGGER.info("Read CSV Speed: " + getReadCSVSpeed(partitionID) + " records/s");
     LOGGER.info("Generate Surrogate Key Speed: " + getGenSurKeySpeed(partitionID) + " records/s");
     LOGGER.info("Sort Key/Write Temp Files Speed: " + getSortKeySpeed(partitionID) + " records/s");
@@ -399,7 +380,6 @@
     loadCsvfilesToDfStartTime = 0;
     loadCsvfilesToDfCostTime = 0;
     dicShuffleAndWriteFileTotalStartTime = 0;
-    dicShuffleAndWriteFileTotalCostTime = 0;
     lruCacheLoadTime = 0;
     totalRecords = 0;
     totalTime = 0;
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 dbac862..8a41594 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
@@ -48,7 +48,6 @@
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 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.datamap.Segment;
 import org.apache.carbondata.core.datastore.FileReader;
@@ -1689,22 +1688,6 @@
   }
 
   /**
-   * This method will check if dictionary and its metadata file exists for a given column
-   *
-   * @param dictionaryColumnUniqueIdentifier unique identifier which contains dbName,
-   *                                         tableName and columnIdentifier
-   * @return
-   */
-  public static boolean isFileExistsForGivenColumn(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    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);
-  }
-
-  /**
    * @param invalidBlockVOForSegmentId
    * @param updateStatusMngr
    * @return
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index 1769e16..0cbb581 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -48,7 +48,6 @@
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_SERIALIZATION_NULL_FORMAT;
-import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_TIMESTAMPFORMAT;
 import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB;
@@ -146,7 +145,6 @@
       case CARBON_CUSTOM_BLOCK_DISTRIBUTION:
       case CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE:
       case CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD:
-      case CARBON_OPTIONS_SINGLE_PASS:
       case ENABLE_VECTOR_READER:
       case ENABLE_UNSAFE_IN_QUERY_EXECUTION:
       case ENABLE_AUTO_LOAD_MERGE:
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 05894ea..0e0aac2 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
@@ -141,14 +141,6 @@
   }
 
   /**
-   * Return absolute path of dictionary file
-   */
-  public static String getExternalDictionaryFilePath(String dictionaryPath, String columnId) {
-    return dictionaryPath + CarbonCommonConstants.FILE_SEPARATOR +
-        getDictionaryFileName(columnId);
-  }
-
-  /**
    * Return metadata path
    */
   public static String getMetadataPath(String tablePath) {
@@ -156,37 +148,6 @@
   }
 
   /**
-   * Return absolute path of dictionary meta file
-   */
-  public static String getExternalDictionaryMetaFilePath(String dictionaryPath, String columnId) {
-    return dictionaryPath + CarbonCommonConstants.FILE_SEPARATOR + columnId + DICTIONARY_META_EXT;
-  }
-
-  /**
-   * Return absolute path of dictionary meta file
-   */
-  public static String getDictionaryMetaFilePath(String tablePath, String columnId) {
-    return getMetadataPath(tablePath) + CarbonCommonConstants.FILE_SEPARATOR + columnId +
-        DICTIONARY_META_EXT;
-  }
-
-  /**
-   * Return sortindex file path based on specified dictionary path
-   */
-  public static String getExternalSortIndexFilePath(String dictionaryPath, String columnId) {
-    return dictionaryPath + CarbonCommonConstants.FILE_SEPARATOR + columnId + SORT_INDEX_EXT;
-  }
-
-  /**
-   * Return sortindex file path for columnId and offset based on specified dictionary path
-   */
-  public static String getExternalSortIndexFilePath(String dictionaryPath, String columnId,
-      long dictOffset) {
-    return dictionaryPath + CarbonCommonConstants.FILE_SEPARATOR +
-        columnId + "_" + dictOffset + SORT_INDEX_EXT;
-  }
-
-  /**
    * return the schema file path
    * @param tablePath path to table files
    * @return schema file path
@@ -585,27 +546,6 @@
   }
 
   /**
-   * Below method will be used to get sort index file present in mentioned folder
-   *
-   * @param sortIndexDir directory where sort index file resides
-   * @param columnUniqueId   columnunique id
-   * @return sort index carbon files
-   */
-  public static CarbonFile[] getSortIndexFiles(CarbonFile sortIndexDir,
-      final String columnUniqueId) {
-    if (null != sortIndexDir) {
-      return sortIndexDir.listFiles(new CarbonFileFilter() {
-        @Override
-        public boolean accept(CarbonFile file) {
-          return file.getName().startsWith(columnUniqueId) && file.getName()
-              .endsWith(SORT_INDEX_EXT);
-        }
-      });
-    }
-    return null;
-  }
-
-  /**
    * Return the carbondata file name
    */
   public static String getCarbonDataFileName(String carbonDataFilePath) {
@@ -693,10 +633,6 @@
     return SEGMENT_PREFIX + value;
   }
 
-  public static String getCarbonIndexFileName(String actualBlockName) {
-    return getShardName(actualBlockName) + INDEX_FILE_EXT;
-  }
-
   /**
    * Unique task name
    *
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
deleted file mode 100644
index b5b6298..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * dictionary writer interface
- */
-public interface CarbonDictionaryWriter extends Closeable {
-  /**
-   * write method that accepts one value at a time
-   * This method can be used when data is huge and memory is les. In that
-   * case data can be stored to a file and an iterator can iterate over it and
-   * pass one value at a time
-   *
-   * @param value unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  void write(String value) throws IOException;
-
-  /**
-   * write method that accepts list of byte arrays as value
-   * This can be used when data is less, then string can be converted
-   * to byte array for each value and added to a list
-   *
-   * @param valueList list of byte array. Each byte array is unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  void write(List<byte[]> valueList) throws IOException;
-
-  void commit() throws IOException;
-}
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
deleted file mode 100644
index a687fd3..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ /dev/null
@@ -1,426 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-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.reader.CarbonDictionaryColumnMetaChunk;
-import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
-import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.HDFSLeaseUtils;
-import org.apache.carbondata.format.ColumnDictionaryChunk;
-import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
-
-import org.apache.log4j.Logger;
-import org.apache.thrift.TBase;
-
-/**
- * This class is responsible for writing the dictionary file and its metadata
- */
-public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
-
-  /**
-   * LOGGER
-   */
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(CarbonDictionaryWriterImpl.class.getName());
-
-  /**
-   * list which will hold values upto maximum of one dictionary chunk size
-   */
-  private List<ByteBuffer> oneDictionaryChunkList;
-
-  /**
-   * Meta object which will hold last segment entry details
-   */
-  private CarbonDictionaryColumnMetaChunk chunkMetaObjectForLastSegmentEntry;
-
-  /**
-   * dictionary file and meta thrift writer
-   */
-  private ThriftWriter dictionaryThriftWriter;
-
-  /**
-   * column identifier
-   */
-  protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  /**
-   * dictionary file path
-   */
-  protected String dictionaryFilePath;
-
-  /**
-   * dictionary metadata file path
-   */
-  protected String dictionaryMetaFilePath;
-
-  /**
-   * start offset of dictionary chunk  for a segment
-   */
-  private long chunk_start_offset;
-
-  /**
-   * end offset of a dictionary chunk for a segment
-   */
-  private long chunk_end_offset;
-
-  /**
-   * total dictionary value record count for one segment
-   */
-  private int totalRecordCount;
-
-  /**
-   * total thrift object chunk count written for one segment
-   */
-  private int chunk_count;
-
-  /**
-   * chunk size for a dictionary file after which data will be written to disk
-   */
-  private int dictionary_one_chunk_size;
-
-  /**
-   * flag to check whether write method is called for first time
-   */
-  private boolean isFirstTime;
-
-  private static final Charset defaultCharset = Charset.forName(
-      CarbonCommonConstants.DEFAULT_CHARSET);
-
-  /**
-   * Constructor
-   *
-   * @param dictionaryColumnUniqueIdentifier column unique identifier
-   */
-  public CarbonDictionaryWriterImpl(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-    this.isFirstTime = true;
-  }
-
-  /**
-   * This method will write the data in thrift format to disk. This method will be guided by
-   * parameter dictionary_one_chunk_size and data will be divided into chunks
-   * based on this parameter
-   *
-   * @param value unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void write(String value) throws IOException {
-    write(value.getBytes(defaultCharset));
-  }
-
-  /**
-   * This method will write the data in thrift format to disk. This method will be guided by
-   * parameter dictionary_one_chunk_size and data will be divided into chunks
-   * based on this parameter
-   *
-   * @param value unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  private void write(byte[] value) throws IOException {
-    if (isFirstTime) {
-      init();
-      isFirstTime = false;
-    }
-
-    if (value.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-      throw new IOException("Dataload failed, String size cannot exceed "
-          + CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes");
-    }
-    // if one chunk size is equal to list size then write the data to file
-    checkAndWriteDictionaryChunkToFile();
-    oneDictionaryChunkList.add(ByteBuffer.wrap(value));
-    totalRecordCount++;
-  }
-
-  /**
-   * This method will write the data in thrift format to disk. This method will not be guided by
-   * parameter dictionary_one_chunk_size and complete data will be written as one chunk
-   *
-   * @param valueList list of byte array. Each byte array is unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void write(List<byte[]> valueList) throws IOException {
-    if (isFirstTime) {
-      init();
-      isFirstTime = false;
-    }
-    for (byte[] value : valueList) {
-      oneDictionaryChunkList.add(ByteBuffer.wrap(value));
-      totalRecordCount++;
-    }
-  }
-
-  /**
-   * write dictionary metadata file and close thrift object
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-    if (null != dictionaryThriftWriter && dictionaryThriftWriter.isOpen()) {
-      try {
-        // if stream is open then only need to write dictionary file.
-        writeDictionaryFile();
-      } finally {
-        // close the thrift writer for dictionary file
-        closeThriftWriter();
-      }
-    }
-  }
-
-  /**
-   * check if the threshold has been reached for the number of
-   * values that can kept in memory and then flush the data to file
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  private void checkAndWriteDictionaryChunkToFile() throws IOException {
-    if (oneDictionaryChunkList.size() >= dictionary_one_chunk_size) {
-      writeDictionaryFile();
-      createChunkList();
-    }
-  }
-
-  /**
-   * This method will serialize the object of dictionary file
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  private void writeDictionaryFile() throws IOException {
-    ColumnDictionaryChunk columnDictionaryChunk = new ColumnDictionaryChunk();
-    columnDictionaryChunk.setValues(oneDictionaryChunkList);
-    writeThriftObject(columnDictionaryChunk);
-  }
-
-  /**
-   * This method will check and created the directory path where dictionary file has to be created
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  private void init() throws IOException {
-    initDictionaryChunkSize();
-    initPaths();
-    boolean dictFileExists = CarbonUtil.isFileExists(this.dictionaryFilePath);
-    if (dictFileExists && CarbonUtil.isFileExists(this.dictionaryMetaFilePath)) {
-      this.chunk_start_offset = CarbonUtil.getFileSize(this.dictionaryFilePath);
-      validateDictionaryFileOffsetWithLastSegmentEntryOffset();
-    } else if (dictFileExists) {
-      FileFactory.getCarbonFile(dictionaryFilePath)
-          .delete();
-    }
-    openThriftWriter(this.dictionaryFilePath);
-    createChunkList();
-  }
-
-  protected void initPaths() {
-    this.dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
-    this.dictionaryMetaFilePath = dictionaryColumnUniqueIdentifier.getDictionaryMetaFilePath();
-  }
-
-  /**
-   * initialize the value of dictionary chunk that can be kept in memory at a time
-   */
-  private void initDictionaryChunkSize() {
-    dictionary_one_chunk_size = CarbonUtil.getDictionaryChunkSize();
-  }
-
-  /**
-   * initialise one dictionary size chunk list and increment chunk count
-   */
-  private void createChunkList() {
-    this.oneDictionaryChunkList = new ArrayList<ByteBuffer>(dictionary_one_chunk_size);
-    chunk_count++;
-  }
-
-  /**
-   * if file already exists then read metadata file and
-   * validate the last entry end offset with file size. If
-   * they are not equal that means some invalid data is present which needs
-   * to be truncated
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  private void validateDictionaryFileOffsetWithLastSegmentEntryOffset() throws IOException {
-    // read last dictionary chunk meta entry from dictionary metadata file
-    chunkMetaObjectForLastSegmentEntry = getChunkMetaObjectForLastSegmentEntry();
-    int bytesToTruncate = 0;
-    if (null != chunkMetaObjectForLastSegmentEntry) {
-      bytesToTruncate =
-          (int) (chunk_start_offset - chunkMetaObjectForLastSegmentEntry.getEnd_offset());
-    }
-    if (bytesToTruncate > 0) {
-      LOGGER.info("some inconsistency in dictionary file for column "
-          + this.dictionaryColumnUniqueIdentifier.getColumnIdentifier());
-      // truncate the dictionary data till chunk meta end offset
-      CarbonFile carbonFile = FileFactory.getCarbonFile(this.dictionaryFilePath);
-      boolean truncateSuccess = carbonFile
-          .truncate(this.dictionaryFilePath, chunkMetaObjectForLastSegmentEntry.getEnd_offset());
-      if (!truncateSuccess) {
-        LOGGER.info("Diction file not truncated successfully for column "
-            + this.dictionaryColumnUniqueIdentifier.getColumnIdentifier());
-      }
-    }
-  }
-
-  /**
-   * This method will write the dictionary metadata file for a given column
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  private void writeDictionaryMetadataFile() throws IOException {
-    // Format of dictionary metadata file
-    // min, max, start offset, end offset and chunk count
-    int min_surrogate_key = 0;
-    int max_surrogate_key = 0;
-    // case 1: first time dictionary writing
-    // previousMax = 0, totalRecordCount = 5, min = 1, max= 5
-    // case2: file already exists
-    // previousMax = 5, totalRecordCount = 10, min = 6, max = 15
-    // case 3: no unique values, total records 0
-    // previousMax = 15, totalRecordCount = 0, min = 15, max = 15
-    // both min and max equal to previous max
-    if (null != chunkMetaObjectForLastSegmentEntry) {
-      if (0 == totalRecordCount) {
-        min_surrogate_key = chunkMetaObjectForLastSegmentEntry.getMax_surrogate_key();
-      } else {
-        min_surrogate_key = chunkMetaObjectForLastSegmentEntry.getMax_surrogate_key() + 1;
-      }
-      max_surrogate_key =
-          chunkMetaObjectForLastSegmentEntry.getMax_surrogate_key() + totalRecordCount;
-    } else {
-      if (totalRecordCount > 0) {
-        min_surrogate_key = 1;
-      }
-      max_surrogate_key = totalRecordCount;
-    }
-    ColumnDictionaryChunkMeta dictionaryChunkMeta =
-        new ColumnDictionaryChunkMeta(min_surrogate_key, max_surrogate_key, chunk_start_offset,
-            chunk_end_offset, chunk_count);
-    try {
-      openThriftWriter(this.dictionaryMetaFilePath);
-      // write dictionary metadata file
-      writeThriftObject(dictionaryChunkMeta);
-      LOGGER.info("Dictionary metadata file written successfully for column "
-          + this.dictionaryColumnUniqueIdentifier.getColumnIdentifier() + " at path "
-          + this.dictionaryMetaFilePath);
-    } finally {
-      closeThriftWriter();
-    }
-  }
-
-  /**
-   * open thrift writer for writing dictionary chunk/meta object
-   *
-   * @param dictionaryFile can be dictionary file name or dictionary metadata file name
-   * @throws IOException if an I/O error occurs
-   */
-  private void openThriftWriter(String dictionaryFile) throws IOException {
-    // create thrift writer instance
-    dictionaryThriftWriter = new ThriftWriter(dictionaryFile, true);
-    // open the file stream
-    try {
-      dictionaryThriftWriter.open();
-    } catch (IOException e) {
-      // Cases to handle
-      // 1. Handle File lease recovery
-      if (HDFSLeaseUtils.checkExceptionMessageForLeaseRecovery(e.getMessage())) {
-        LOGGER.error("Lease recovery exception encountered for file: " + dictionaryFile, e);
-        boolean leaseRecovered = HDFSLeaseUtils.recoverFileLease(dictionaryFile);
-        if (leaseRecovered) {
-          // try to open output stream again after recovering the lease on file
-          dictionaryThriftWriter.open();
-        } else {
-          throw e;
-        }
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  /**
-   * This method will write the thrift object to a file
-   *
-   * @param dictionaryThriftObject can be dictionary thrift object or dictionary metadata
-   *                               thrift object
-   * @throws IOException if an I/O error occurs
-   */
-  private void writeThriftObject(TBase dictionaryThriftObject) throws IOException {
-    dictionaryThriftWriter.write(dictionaryThriftObject);
-  }
-
-  /**
-   * close dictionary thrift writer
-   */
-  private void closeThriftWriter() throws IOException {
-    if (null != dictionaryThriftWriter) {
-      dictionaryThriftWriter.close();
-    }
-  }
-
-  /**
-   * This method will read the dictionary chunk metadata thrift object for last entry
-   *
-   * @return last entry of dictionary meta chunk
-   * @throws IOException if an I/O error occurs
-   */
-  private CarbonDictionaryColumnMetaChunk getChunkMetaObjectForLastSegmentEntry()
-      throws IOException {
-    CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
-    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
-    try {
-      // read the last segment entry for dictionary metadata
-      carbonDictionaryColumnMetaChunk =
-          columnMetadataReaderImpl.readLastEntryOfDictionaryMetaChunk();
-    } finally {
-      // Close metadata reader
-      columnMetadataReaderImpl.close();
-    }
-    return carbonDictionaryColumnMetaChunk;
-  }
-
-  /**
-   * @return
-   */
-  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
-    return new CarbonDictionaryMetadataReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  @Override
-  public void commit() throws IOException {
-    if (null != dictionaryThriftWriter && dictionaryThriftWriter.isOpen()) {
-      this.chunk_end_offset = CarbonUtil.getFileSize(this.dictionaryFilePath);
-      writeDictionaryMetadataFile();
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriter.java
deleted file mode 100644
index c84e1db..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriter.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.io.Closeable;
-import java.util.List;
-
-/**
- * Interface for writing the dictionary sort index and sort index revers data.
- */
-public interface CarbonDictionarySortIndexWriter extends Closeable {
-
-  /**
-   * The method is used write the dictionary sortIndex data to columns
-   * sortedIndex file in thrif format.
-   *
-   * @param sortIndexList list of sortIndex
-   */
-  void writeSortIndex(List<Integer> sortIndexList);
-
-  /**
-   * The method is used write the dictionary sortIndexInverted data to columns
-   * sortedIndex file in thrif format.
-   *
-   * @param invertedSortIndexList list of  sortIndexInverted
-   */
-  void writeInvertedSortIndex(List<Integer> invertedSortIndexList);
-
-}
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
deleted file mode 100644
index 4824a00..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-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.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.writer.ThriftWriter;
-import org.apache.carbondata.format.ColumnSortInfo;
-
-import org.apache.log4j.Logger;
-
-/**
- * The class responsible for writing the dictionary/column sort index and sort index inverted data
- * in the thrift format
- */
-public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySortIndexWriter {
-
-  /**
-   * column name
-   */
-  protected DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  /**
-   * Path of dictionary sort index file for which the sortIndex to be written
-   */
-  protected String sortIndexFilePath;
-  /**
-   * Instance of thrift writer to write the data
-   */
-  private ThriftWriter sortIndexThriftWriter;
-
-  /**
-   * Column sort info thrift instance.
-   */
-  private ColumnSortInfo columnSortInfo = new ColumnSortInfo();
-
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(CarbonDictionarySortIndexWriterImpl.class.getName());
-
-  /**
-   * @param dictionaryColumnUniqueIdentifier      column unique identifier
-   */
-  public CarbonDictionarySortIndexWriterImpl(
-      final DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
-  }
-
-  /**
-   * The method is used populate the dictionary sortIndex data to columnSortInfo
-   * in thrif format.
-   *
-   * @param sortIndexList list of sortIndex
-   * @throws IOException In Case of any I/O errors occurs.
-   */
-  @Override
-  public void writeSortIndex(List<Integer> sortIndexList) {
-    columnSortInfo.setSort_index(sortIndexList);
-  }
-
-  /**
-   * The method is used populate the dictionary Inverted sortIndex data to columnSortInfo
-   * in thrif format.
-   *
-   * @param invertedSortIndexList list of  sortIndexInverted
-   * @throws IOException In Case of any I/O errors occurs.
-   */
-  @Override
-  public void writeInvertedSortIndex(List<Integer> invertedSortIndexList) {
-    columnSortInfo.setSort_index_inverted(invertedSortIndexList);
-  }
-
-  /**
-   * Initialize the sortIndexFilePath and open writing stream
-   * for dictionary sortIndex file thrif writer
-   * write the column sort info to the store when both sort index  and sort index
-   * inverted are populated.
-   * existing sort index file has to be overwritten with new sort index data
-   * columnSortInfo having null sortIndex and invertedSortIndex will not be written
-   */
-  private void writeColumnSortInfo() throws IOException {
-    boolean isNotNull =
-        null != columnSortInfo.getSort_index() && null != columnSortInfo.sort_index_inverted;
-    if (isNotNull) {
-      initPath();
-      String folderContainingFile = CarbonTablePath.getFolderContainingFile(this.sortIndexFilePath);
-      boolean created = CarbonUtil.checkAndCreateFolder(folderContainingFile);
-      if (!created) {
-        LOGGER.error("Database metadata folder creation status :: " + created);
-        throw new IOException("Failed to created database metadata folder");
-      }
-      try {
-
-        this.sortIndexThriftWriter = new ThriftWriter(this.sortIndexFilePath, false);
-        this.sortIndexThriftWriter.open();
-        sortIndexThriftWriter.write(columnSortInfo);
-      } catch (IOException ie) {
-        LOGGER.error(
-            "problem while writing the dictionary sort index file.", ie);
-        throw new IOException("problem while writing the dictionary sort index file.", ie);
-      } finally {
-        if (null != sortIndexThriftWriter) {
-          this.sortIndexThriftWriter.close();
-        }
-        this.sortIndexFilePath = null;
-      }
-    }
-  }
-
-  protected void initPath() {
-    String dictionaryFilePath = dictionaryColumnUniqueIdentifier.getDictionaryFilePath();
-    long dictOffset = CarbonUtil.getFileSize(dictionaryFilePath);
-    this.sortIndexFilePath = dictionaryColumnUniqueIdentifier.getSortIndexFilePath(dictOffset);
-    cleanUpOldSortIndex(dictionaryFilePath);
-  }
-
-  /**
-   * It cleans up old unused sortindex file
-   *
-   * @param dictPath
-   */
-  protected void cleanUpOldSortIndex(String dictPath) {
-    CarbonFile dictFile = FileFactory.getCarbonFile(dictPath);
-    CarbonFile[] files = CarbonTablePath.getSortIndexFiles(dictFile.getParentFile(),
-        dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId());
-    int maxTime;
-    try {
-      maxTime = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.MAX_QUERY_EXECUTION_TIME));
-    } catch (NumberFormatException e) {
-      maxTime = CarbonCommonConstants.DEFAULT_MAX_QUERY_EXECUTION_TIME;
-    }
-    if (null != files) {
-      Arrays.sort(files, new Comparator<CarbonFile>() {
-        @Override
-        public int compare(CarbonFile o1, CarbonFile o2) {
-          return o1.getName().compareTo(o2.getName());
-        }
-      });
-      for (int i = 0; i < files.length - 1; i++) {
-        long difference = System.currentTimeMillis() - files[i].getLastModifiedTime();
-        long minutesElapsed = (difference / (1000 * 60));
-        if (minutesElapsed > maxTime) {
-          if (!files[i].delete()) {
-            LOGGER.warn("Failed to delete sortindex file." + files[i].getAbsolutePath());
-          } else {
-            LOGGER.info("Sort index file is deleted." + files[i].getAbsolutePath());
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated
-   * with it. If the stream is already closed then invoking this
-   * method has no effect.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-    writeColumnSortInfo();
-    if (null != sortIndexThriftWriter) {
-      sortIndexThriftWriter.close();
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfo.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfo.java
deleted file mode 100644
index 93f2a9e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfo.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.util.List;
-
-/**
- * Model to hold the sortIndex and sortIndexInverted data
- */
-public class CarbonDictionarySortInfo {
-  /**
-   * Sort index after members are sorted
-   */
-  private List<Integer> sortIndex;
-  /**
-   * inverted sort index to get the member
-   */
-  private List<Integer> sortIndexInverted;
-
-  /**
-   * The constructor to instantiate the CarbonDictionarySortInfo object
-   * with sortIndex and sortInverted Index data
-   *
-   * @param sortIndex
-   * @param sortIndexInverted
-   */
-  public CarbonDictionarySortInfo(List<Integer> sortIndex, List<Integer> sortIndexInverted) {
-    this.sortIndex = sortIndex;
-    this.sortIndexInverted = sortIndexInverted;
-  }
-
-  /**
-   * return list of sortIndex
-   *
-   * @return
-   */
-  public List<Integer> getSortIndex() {
-    return sortIndex;
-  }
-
-  /**
-   * returns list of sortindexinverted
-   *
-   * @return
-   */
-  public List<Integer> getSortIndexInverted() {
-    return sortIndexInverted;
-  }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparator.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparator.java
deleted file mode 100644
index bbe7659..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparator.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryChunksWrapper;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-
-import org.apache.commons.lang.ArrayUtils;
-
-/**
- * The class prepares the column sort info ie sortIndex
- * and inverted sort index info
- */
-public class CarbonDictionarySortInfoPreparator {
-
-  /**
-   * The method returns the column Sort Info
-   *
-   * @param newDistinctValues new distinct value to be added
-   * @param dictionary        old distinct values
-   * @param dataType          DataType of columns
-   * @return CarbonDictionarySortInfo returns the column Sort Info
-   */
-  public CarbonDictionarySortInfo getDictionarySortInfo(List<String> newDistinctValues,
-      Dictionary dictionary, DataType dataType) {
-    CarbonDictionarySortModel[] dictionarySortModels =
-        prepareDictionarySortModels(newDistinctValues, dictionary, dataType);
-    return createColumnSortInfo(dictionarySortModels);
-  }
-
-  /**
-   * The method prepares the sort_index and sort_index_inverted data
-   *
-   * @param dictionarySortModels
-   */
-  private CarbonDictionarySortInfo createColumnSortInfo(
-      CarbonDictionarySortModel[] dictionarySortModels) {
-
-    //Sort index after members are sorted
-    int[] sortIndex;
-    //inverted sort index to get the member
-    int[] sortIndexInverted;
-
-    Arrays.sort(dictionarySortModels);
-    sortIndex = new int[dictionarySortModels.length];
-    sortIndexInverted = new int[dictionarySortModels.length];
-
-    for (int i = 0; i < dictionarySortModels.length; i++) {
-      CarbonDictionarySortModel dictionarySortModel = dictionarySortModels[i];
-      sortIndex[i] = dictionarySortModel.getKey();
-      // the array index starts from 0 therefore -1 is done to avoid wastage
-      // of 0th index in array and surrogate key starts from 1 there 1 is added to i
-      // which is a counter starting from 0
-      sortIndexInverted[dictionarySortModel.getKey() - 1] = i + 1;
-    }
-    dictionarySortModels = null;
-    List<Integer> sortIndexList = convertToList(sortIndex);
-    List<Integer> sortIndexInvertedList = convertToList(sortIndexInverted);
-    return new CarbonDictionarySortInfo(sortIndexList, sortIndexInvertedList);
-  }
-
-  /**
-   * The method converts the int[] to List<Integer>
-   *
-   * @param data
-   * @return
-   */
-  private List<Integer> convertToList(int[] data) {
-    Integer[] wrapperType = ArrayUtils.toObject(data);
-    return Arrays.asList(wrapperType);
-  }
-
-  /**
-   * The method returns the array of CarbonDictionarySortModel
-   *
-   * @param distinctValues new distinct values
-   * @param dictionary The wrapper wraps the list<list<bye[]>> and provide the
-   *                   iterator to retrieve the chunks members.
-   * @param dataType   DataType of columns
-   * @return CarbonDictionarySortModel[] CarbonDictionarySortModel[] the model
-   * CarbonDictionarySortModel contains the  member's surrogate and
-   * its byte value
-   */
-  private CarbonDictionarySortModel[] prepareDictionarySortModels(List<String> distinctValues,
-      Dictionary dictionary, DataType dataType) {
-    CarbonDictionarySortModel[] dictionarySortModels = null;
-    //The wrapper wraps the list<list<bye[]>> and provide the iterator to
-    // retrieve the chunks members.
-    int surrogate = 1;
-    if (null != dictionary) {
-      DictionaryChunksWrapper dictionaryChunksWrapper = dictionary.getDictionaryChunks();
-      dictionarySortModels =
-          new CarbonDictionarySortModel[dictionaryChunksWrapper.getSize() + distinctValues.size()];
-      while (dictionaryChunksWrapper.hasNext()) {
-        dictionarySortModels[surrogate - 1] =
-            createDictionarySortModel(surrogate, dataType, dictionaryChunksWrapper.next());
-        surrogate++;
-      }
-    } else {
-      dictionarySortModels = new CarbonDictionarySortModel[distinctValues.size()];
-    }
-    // for new distinct values
-    Iterator<String> distinctValue = distinctValues.iterator();
-    while (distinctValue.hasNext()) {
-      dictionarySortModels[surrogate - 1] = createDictionarySortModel(surrogate, dataType,
-          distinctValue.next().getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-      surrogate++;
-    }
-    return dictionarySortModels;
-  }
-
-  /**
-   *
-   * @param surrogate
-   * @param dataType
-   * @param value member value
-   * @return CarbonDictionarySortModel
-   */
-  private CarbonDictionarySortModel createDictionarySortModel(int surrogate, DataType dataType,
-      byte[] value) {
-    String memberValue = new String(value, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    return new CarbonDictionarySortModel(surrogate, dataType, memberValue);
-  }
-}
-
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java b/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
deleted file mode 100644
index 3d2bc05..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModel.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-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.util.CarbonUtil;
-
-/**
- * Dictionary sort model class holds the member byte value and corresponding key value.
- */
-public class CarbonDictionarySortModel implements Comparable<CarbonDictionarySortModel> {
-
-  /**
-   * Surrogate key
-   */
-  private int key;
-
-  /**
-   * member value in bytes
-   */
-  private String memberValue;
-
-  /**
-   * member dataType
-   */
-  private DataType dataType;
-
-  /**
-   * Constructor to init the dictionary sort model
-   *
-   * @param key
-   * @param dataType
-   * @param memberValue
-   */
-  public CarbonDictionarySortModel(int key, DataType dataType, String memberValue) {
-    this.key = key;
-    this.dataType = dataType;
-    this.memberValue = memberValue;
-  }
-
-  /**
-   * Compare
-   */
-  @Override
-  public int compareTo(CarbonDictionarySortModel o) {
-    if (dataType == DataTypes.SHORT ||
-        dataType == DataTypes.INT ||
-        dataType == DataTypes.LONG ||
-        dataType == DataTypes.DOUBLE) {
-      Double d1 = null;
-      Double d2 = null;
-      try {
-        d1 = new Double(memberValue);
-      } catch (NumberFormatException e) {
-        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
-          return -1;
-        }
-        return 1;
-      }
-      try {
-        d2 = new Double(o.memberValue);
-      } catch (NumberFormatException e) {
-        return -1;
-      }
-      return d1.compareTo(d2);
-    } else if (DataTypes.isDecimal(dataType)) {
-      java.math.BigDecimal val1 = null;
-      java.math.BigDecimal val2 = null;
-      try {
-        val1 = new java.math.BigDecimal(memberValue);
-      } catch (NumberFormatException e) {
-        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
-          return -1;
-        }
-        return 1;
-      }
-      try {
-        val2 = new java.math.BigDecimal(o.memberValue);
-      } catch (NumberFormatException e) {
-        return -1;
-      }
-      return val1.compareTo(val2);
-    } else if (dataType == DataTypes.DATE || dataType == DataTypes.TIMESTAMP) {
-      String format = CarbonUtil.getFormatFromProperty(dataType);
-      SimpleDateFormat parser = new SimpleDateFormat(format);
-      Date date1 = null;
-      Date date2 = null;
-      try {
-        date1 = parser.parse(memberValue);
-      } catch (ParseException e) {
-        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(o.memberValue)) {
-          return -1;
-        }
-        return 1;
-      }
-      try {
-        date2 = parser.parse(o.memberValue);
-      } catch (ParseException e) {
-        return -1;
-      }
-      return date1.compareTo(date2);
-    } else {
-      return this.memberValue.compareTo(o.memberValue);
-    }
-  }
-
-  /**
-   * @see Object#hashCode()
-   */
-  @Override
-  public int hashCode() {
-    return ((memberValue == null) ? 0 : memberValue.hashCode());
-  }
-
-  /**
-   * @see Object#equals(Object)
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof CarbonDictionarySortModel) {
-      if (this == obj) {
-        return true;
-      }
-      CarbonDictionarySortModel other = (CarbonDictionarySortModel) obj;
-      if (memberValue == null) {
-        if (other.memberValue != null) {
-          return false;
-        }
-      } else if (!this.memberValue.equals(other.memberValue)) {
-        return false;
-      }
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * return the surrogate of the member
-   *
-   * @return
-   */
-  public int getKey() {
-    return key;
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java b/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
deleted file mode 100644
index deb1a29..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/CacheProviderTest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.cache.dictionary.ForwardDictionaryCache;
-import org.apache.carbondata.core.cache.dictionary.ReverseDictionaryCache;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test class to test dictionary cache functionality
- */
-public class CacheProviderTest {
-
-  @Before public void setUp() throws Exception {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE, "20");
-  }
-
-  @Test public void getInstance() throws Exception {
-    // get cache provider instance
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    // assert for cache provider instance
-    assertTrue(cacheProvider instanceof CacheProvider);
-  }
-
-  @Test public void createCache() throws Exception {
-    // get cache provider instance
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictionaryCache =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
-    // assert that dictionary cache is an instance of Forward dictionary cache
-    assertTrue(dictionaryCache instanceof ForwardDictionaryCache);
-    assertFalse(dictionaryCache instanceof ReverseDictionaryCache);
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> reverseDictionaryCache =
-        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
-    // assert that dictionary cache is an instance of Reverse dictionary cache
-    assertTrue(reverseDictionaryCache instanceof ReverseDictionaryCache);
-    assertFalse(reverseDictionaryCache instanceof ForwardDictionaryCache);
-    cacheProvider.dropAllCache();
-  }
-}
\ No newline at end of file
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 5263faf..e69de29 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
@@ -1,181 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
-
-public class AbstractDictionaryCacheTest {
-
-  protected static final String PROPERTY_FILE_NAME = "carbonTest.properties";
-
-  protected CarbonTableIdentifier carbonTableIdentifier;
-
-  protected AbsoluteTableIdentifier identifier;
-
-  protected String databaseName;
-
-  protected String tableName;
-
-  protected String carbonStorePath;
-
-  protected Properties props;
-
-  protected List<String> dataSet1;
-
-  protected List<String> dataSet2;
-
-  protected List<String> dataSet3;
-
-  protected String[] columnIdentifiers;
-
-  /**
-   * this method will delete the folders recursively
-   *
-   * @param f
-   */
-  protected static void deleteRecursiveSilent(CarbonFile f) {
-    if (f.isDirectory()) {
-      if (f.listFiles() != null) {
-        for (CarbonFile c : f.listFiles()) {
-          deleteRecursiveSilent(c);
-        }
-      }
-    }
-    if (f.exists() && !f.delete()) {
-      return;
-    }
-  }
-
-  /**
-   * prepare the dataset required for running test cases
-   */
-  protected void prepareDataSet() {
-    dataSet1 = Arrays.asList("a", "b", "c");
-    dataSet2 = Arrays.asList("d", "e", "f");
-    dataSet3 = Arrays.asList("b", "c", "a", "d");
-  }
-
-  /**
-   * This method will remove the column identifiers from lru cache
-   */
-  protected void removeKeyFromLRUCache(Cache cacheObject) {
-    for (int i = 0; i < columnIdentifiers.length; i++) {
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          createDictionaryColumnUniqueIdentifier(columnIdentifiers[i]);
-      cacheObject.invalidate(dictionaryColumnUniqueIdentifier);
-    }
-  }
-
-  protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
-      String columnId) {
-	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.STRING);
-    return new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
-        DataTypes.STRING);
-  }
-
-  /**
-   * this method will delete the store path
-   */
-  protected void deleteStorePath() {
-    CarbonFile carbonFile = FileFactory.getCarbonFile(this.carbonStorePath);
-    deleteRecursiveSilent(carbonFile);
-  }
-
-  /**
-   * write dictionary data
-   *
-   * @param data
-   * @throws IOException
-   */
-  protected void prepareWriterAndWriteData(List<String> data, String columnId)
-      throws IOException {
-	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    CarbonDictionaryWriter carbonDictionaryWriter =
-        new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-    CarbonUtil.checkAndCreateFolder(CarbonTablePath.getMetadataPath(identifier.getTablePath()));
-    List<byte[]> valueList = convertStringListToByteArray(data);
-    try {
-      carbonDictionaryWriter.write(valueList);
-    } finally {
-      carbonDictionaryWriter.close();
-      carbonDictionaryWriter.commit();
-    }
-  }
-
-  /**
-   * this method will convert list of string to list of byte array
-   */
-  protected List<byte[]> convertStringListToByteArray(List<String> valueList) {
-    List<byte[]> byteArrayList = new ArrayList<>(valueList.size());
-    for (String value : valueList) {
-      byteArrayList.add(value.getBytes(Charset.defaultCharset()));
-    }
-    return byteArrayList;
-  }
-
-  /**
-   * this method will read the property file for required details
-   * like dbName, tableName, etc
-   */
-  protected void init() {
-    InputStream in = null;
-    props = new Properties();
-    try {
-      URL url = getClass().getClassLoader().getResource(PROPERTY_FILE_NAME);
-      if (null != url){
-
-        URI uri = url.toURI();
-        File file = new File(uri);
-        in = new FileInputStream(file);
-        props.load(in);
-      }
-    } catch (IOException e) {
-      e.printStackTrace();
-    } catch (URISyntaxException e) {
-      e.printStackTrace();
-    } finally {
-      CarbonUtil.closeStreams(in);
-    }
-  }
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIteratorTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIteratorTest.java
deleted file mode 100644
index ccd7c0e..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIteratorTest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.carbondata.format.ColumnDictionaryChunk;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-public class ColumnDictionaryChunkIteratorTest {
-
-  private static ColumnDictionaryChunkIterator columnDictionaryChunkIterator;
-  private static List<byte[]> expectedResult;
-
-  @BeforeClass public static void setUp() {
-    ColumnDictionaryChunk columnDictionaryChunk1 = new ColumnDictionaryChunk();
-    ByteBuffer byteBuffer3 = ByteBuffer.wrap("c".getBytes());
-    ByteBuffer byteBuffer4 = ByteBuffer.wrap("d".getBytes());
-    columnDictionaryChunk1.setValues(new ArrayList<ByteBuffer>());
-    ColumnDictionaryChunk columnDictionaryChunk2 = new ColumnDictionaryChunk();
-    columnDictionaryChunk2.setValues(Arrays.asList(byteBuffer3, byteBuffer4));
-    expectedResult = prepareExpectedData();
-    columnDictionaryChunkIterator = new ColumnDictionaryChunkIterator(
-        Arrays.asList(columnDictionaryChunk1, columnDictionaryChunk2));
-  }
-
-  private static List<byte[]> prepareExpectedData() {
-    List<byte[]> chunks = new ArrayList<>();
-    chunks.add("c".getBytes());
-    chunks.add("d".getBytes());
-    return chunks;
-  }
-
-  @Test public void testNext() throws Exception {
-    List<byte[]> actual = new ArrayList<>();
-    while (columnDictionaryChunkIterator.hasNext()) {
-      actual.add(columnDictionaryChunkIterator.next());
-    }
-    for (int i = 0; i < actual.size(); i++) {
-      assertThat(expectedResult.get(i), is(equalTo(actual.get(i))));
-    }
-  }
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
deleted file mode 100644
index 172dfa4..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfoTest.java
+++ /dev/null
@@ -1,592 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.hasItems;
-import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.CoreMatchers.nullValue;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-public class ColumnDictionaryInfoTest {
-
-  private ColumnDictionaryInfo columnDictionaryInfo;
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionary() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    List<String> evaluateResultList = Arrays.asList("china", "france");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2, 3, 4, 5, 6, 7));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 64, 78, 85, 35, 76, 76, 36, 33 },
-        new byte[] { 98, 114, 97, 122, 105, 108 }, new byte[] { 99, 97, 110, 97, 100, 97 },
-        new byte[] { 99, 104, 105, 110, 97 }, new byte[] { 102, 114, 97, 110, 99, 101 },
-        new byte[] { 117, 107 }, new byte[] { 117, 117, 97 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    assertThat(surrogates.size(), is(equalTo(2)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(4);
-    expectedSurrogates.add(5);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithZeroSurrogate() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    List<String> evaluateResultList = Arrays.asList("china", "france");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2, 3, 4, 5, 6, 7));
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 1;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(0);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithNullValue() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    List<String> evaluateResultList = Arrays.asList("@NU#LL$!");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(2, 3, 1));
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 1;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(1);
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithTypeException() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
-
-    List<String> evaluateResultList = Arrays.asList("china", "france");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2, 3, 4, 5, 6, 7));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 64, 78, 85, 35, 76, 76, 36, 33 },
-        new byte[] { 98, 114, 97, 122, 105, 108 }, new byte[] { 99, 97, 110, 97, 100, 97 },
-        new byte[] { 99, 104, 105, 110, 97 }, new byte[] { 102, 114, 97, 110, 99, 101 },
-        new byte[] { 117, 107 }, new byte[] { 117, 117, 97 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 2;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(4);
-    expectedSurrogates.add(5);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithDoubleType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.DOUBLE);
-
-    List<String> evaluateResultList = Arrays.asList("15999");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 49, 53, 57, 57, 57 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 1;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(1);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithIntType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
-
-    List<String> evaluateResultList = Arrays.asList("998");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 57, 57, 56 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 1;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(1);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithDecimalType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.createDefaultDecimalType());
-
-    List<String> evaluateResultList = Arrays.asList("150011.550");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(2, 3, 1));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 64, 78, 85, 35, 76, 76, 36, 33 },
-        new byte[] { 49, 53, 48, 48, 48, 49, 46, 50, 53, 54 },
-        new byte[] { 49, 53, 48, 48, 49, 49, 46, 53, 53, 48 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 1;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(3);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testGetIncrementalSurrogateKeyFromDictionaryWithLongType() {
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.LONG);
-
-    List<String> evaluateResultList = Arrays.asList("1500115505555");
-    List<byte[]> byteValuesOfFilterMembers = convertListElementsIntoByteArray(evaluateResultList);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(2, 3, 1));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 64, 78, 85, 35, 76, 76, 36, 33 },
-        new byte[] { 49, 53, 48, 48, 48, 49, 50, 53, 54, 52, 52, 52, 52 },
-        new byte[] { 49, 53, 48, 48, 49, 49, 53, 53, 48, 53, 53, 53, 53 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    List<Integer> surrogates = new ArrayList<>();
-    columnDictionaryInfo
-        .getIncrementalSurrogateKeyFromDictionary(byteValuesOfFilterMembers, surrogates);
-
-    int expectedSize = 1;
-    assertThat(surrogates.size(), is(equalTo(expectedSize)));
-
-    List<Integer> expectedSurrogates = new ArrayList<>();
-    expectedSurrogates.add(3);
-
-    assertThat(surrogates, is(equalTo(expectedSurrogates)));
-  }
-
-  @Test public void testAddDictionaryChunkEmpty() {
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 1;
-      }
-    };
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
-
-    List<byte[]> newDictionaryChunk = Arrays.asList(new byte[] { 1, 2, 3, 4 });
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk);
-
-    List<List<byte[]>> expectedDictionaryChunks = new CopyOnWriteArrayList<>();
-    expectedDictionaryChunks.add(newDictionaryChunk);
-
-    assertThat(columnDictionaryInfo.dictionaryChunks, is(equalTo(expectedDictionaryChunks)));
-  }
-
-  @Test public void testAddDictionaryChunkAppend() {
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 1;
-      }
-    };
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
-
-    List<byte[]> newDictionaryChunk1 = Arrays.asList(new byte[] { 1, 2, 3, 4 });
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk1);
-
-    List<byte[]> newDictionaryChunk2 = Arrays.asList(new byte[] { 5, 6, 7, 8 });
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk2);
-
-    List<List<byte[]>> expectedDictionaryChunks = new CopyOnWriteArrayList<>();
-    expectedDictionaryChunks.add(newDictionaryChunk1);
-    expectedDictionaryChunks.add(newDictionaryChunk2);
-
-    assertThat(columnDictionaryInfo.dictionaryChunks, is(equalTo(expectedDictionaryChunks)));
-  }
-
-  @Test public void addDictionaryChunkWithHugeChunkSize() {
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10;
-      }
-    };
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
-
-    List<byte[]> newDictionaryChunk1 = new ArrayList<>(Arrays.asList(new byte[] { 1, 2, 3, 4 }));
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk1);
-
-    List<byte[]> newDictionaryChunk2 = new ArrayList<>(Arrays.asList(new byte[] { 5, 6, 7, 8 }));
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk2);
-
-    List<List<byte[]>> expectedDictionaryChunks = new CopyOnWriteArrayList<>();
-    expectedDictionaryChunks.add(newDictionaryChunk1);
-
-    assertThat(columnDictionaryInfo.dictionaryChunks, is(equalTo(expectedDictionaryChunks)));
-  }
-
-  @Test public void addDictionaryChunkWithSplitChunks() {
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 2;
-      }
-    };
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
-
-    List<byte[]> newDictionaryChunk1 = new ArrayList<>(Arrays.asList(new byte[] { 1, 2, 3, 4 }));
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk1);
-
-    List<byte[]> newDictionaryChunk2 =
-        new ArrayList<>(Arrays.asList(new byte[] { 5, 6, 7 }, new byte[] { 8, 9, 10 }));
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk2);
-
-    byte[][] expectedResult = {{ 1, 2, 3, 4 }, { 5, 6, 7 }, { 8, 9, 10 }};
-    assertThat(columnDictionaryInfo.dictionaryChunks.get(0),
-        hasItems(expectedResult[0], expectedResult[1]));
-    assertThat(columnDictionaryInfo.dictionaryChunks.get(1), hasItems(expectedResult[2]));
-  }
-
-  @Test public void testGtSortedIndexWithMinimumSurrogateKey() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
-
-    final int result = columnDictionaryInfo.getSortedIndex(0);
-
-    int expectedResult = -1;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testGtSortedIndexWithMaximumSurrogateKey() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
-
-    final int result = columnDictionaryInfo.getSortedIndex(4);
-
-    int expectedResult = -1;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testGtSortedIndexWithSurrogateKey() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
-
-    final int result = columnDictionaryInfo.getSortedIndex(2);
-
-    int expectedResult = 2;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testGetSizeOfLastDictionaryChunkWithDictionaryChunkZero() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    final int result = columnDictionaryInfo.getSizeOfLastDictionaryChunk();
-
-    int expectedResult = 0;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testGetSizeOfLastDictionaryChunk() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.dictionaryChunks = new CopyOnWriteArrayList<>();
-
-    List<byte[]> newDictionaryChunk1 = new ArrayList<>(Arrays.asList(new byte[] { 1, 2, 3, 4 }));
-
-    columnDictionaryInfo.addDictionaryChunk(newDictionaryChunk1);
-
-    final int result = columnDictionaryInfo.getSizeOfLastDictionaryChunk();
-
-    int expectedResult = 1;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testGetDictionaryValueFromSortedIndexWithMinimumSurrogateKey() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
-
-    final String result = columnDictionaryInfo.getDictionaryValueFromSortedIndex(0);
-
-    assertThat(result, is(nullValue()));
-  }
-
-  @Test public void testGetDictionaryValueFromSortedIndexWithMaximumSurrogateKey() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(1, 2, 3));
-
-    final String result = columnDictionaryInfo.getDictionaryValueFromSortedIndex(4);
-
-    assertThat(result, is(nullValue()));
-  }
-
-  @Test public void testGetDictionaryValueFromSortedIndex() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    columnDictionaryInfo.setSortReverseOrderIndex(Arrays.asList(0, 1, 2, 3));
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2, 3));
-
-    final String result = columnDictionaryInfo.getDictionaryValueFromSortedIndex(1);
-
-    assertThat(result, is(nullValue()));
-  }
-
-  @Test
-  public void testGetSurrogateKey() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-
-    byte[] value = convertListElementsIntoByteArray(Arrays.asList("china")).get(0);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2, 3, 4, 5, 6, 7));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 64, 78, 85, 35, 76, 76, 36, 33 },
-        new byte[] { 98, 114, 97, 122, 105, 108 }, new byte[] { 99, 97, 110, 97, 100, 97 },
-        new byte[] { 99, 104, 105, 110, 97 }, new byte[] { 102, 114, 97, 110, 99, 101 },
-        new byte[] { 117, 107 }, new byte[] { 117, 117, 97 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    int result = columnDictionaryInfo.getSurrogateKey(value);
-
-    int expectedResult = 4;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  @Test
-  public void testGetSurrogateKeyWithIntType() {
-
-    columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
-
-    byte[] value = convertListElementsIntoByteArray(Arrays.asList("998")).get(0);
-
-    new MockUp<CarbonUtil>() {
-
-      @Mock public int getDictionaryChunkSize() {
-        return 10000;
-      }
-    };
-
-    columnDictionaryInfo.setSortOrderIndex(Arrays.asList(1, 2));
-
-    List<byte[]> chunks = Arrays.asList(new byte[] { 57, 57, 56 });
-
-    List<List<byte[]>> dictionaryChunks = new CopyOnWriteArrayList<>();
-    dictionaryChunks.add(chunks);
-
-    columnDictionaryInfo.dictionaryChunks = dictionaryChunks;
-
-    int result = columnDictionaryInfo.getSurrogateKey(value);
-
-    int expectedResult = 1;
-    assertThat(result, is(equalTo(expectedResult)));
-  }
-
-  private List<byte[]> convertListElementsIntoByteArray(List<String> stringList) {
-    List<byte[]> byteValuesOfFilterMembers = new ArrayList<>(stringList.size());
-    for (int i = 0; i < stringList.size(); i++) {
-      byte[] keyData =
-          stringList.get(i).getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-      byteValuesOfFilterMembers.add(keyData);
-    }
-    return byteValuesOfFilterMembers;
-  }
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnReverseDictionaryInfoTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnReverseDictionaryInfoTest.java
deleted file mode 100644
index 2d1c7a0..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ColumnReverseDictionaryInfoTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.util.Arrays;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.assertThat;
-
-public class ColumnReverseDictionaryInfoTest {
-
-  private static ColumnReverseDictionaryInfo columnReverseDictionaryInfo;
-
-  @BeforeClass public static void setUp() {
-    columnReverseDictionaryInfo = new ColumnReverseDictionaryInfo();
-    columnReverseDictionaryInfo.addDictionaryChunk(Arrays.asList("a".getBytes()));
-    columnReverseDictionaryInfo.addDictionaryChunk(Arrays.asList("b".getBytes()));
-  }
-
-  @Test public void testToGetSurrogateKey() {
-    int key1 = columnReverseDictionaryInfo.getSurrogateKey("a".getBytes());
-    int key2 = columnReverseDictionaryInfo.getSurrogateKey("b".getBytes());
-    int[] surrogateKey = { key1, key2 };
-    int[] expectedKeys = { 1, 2 };
-    assertThat(surrogateKey, is(equalTo(expectedKeys)));
-  }
-
-  @Test public void testToGetSurrogateKeyForInvalidKey() {
-    int key = columnReverseDictionaryInfo.getSurrogateKey("c".getBytes());
-    int expectedKey = -1;
-    assertThat(key, is(equalTo(expectedKey)));
-  }
-}
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
deleted file mode 100644
index e5d0975..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.reader.CarbonDictionaryReaderImpl;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
-import org.apache.carbondata.format.ColumnDictionaryChunk;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class DictionaryCacheLoaderImplTest {
-
-  private static DictionaryCacheLoaderImpl dictionaryCacheLoader;
-  private static DictionaryInfo dictionaryInfo;
-  private static ColumnIdentifier columnIdentifier;
-  private static DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  @BeforeClass public static void setUp() {
-    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("db", "table1", "1");
-    AbsoluteTableIdentifier absoluteTableIdentifier = AbsoluteTableIdentifier.from("/tmp",
-        carbonTableIdentifier);
-    Map<String, String> columnProperties = new HashMap<>();
-    columnProperties.put("prop1", "value1");
-    columnProperties.put("prop2", "value2");
-    columnIdentifier = new ColumnIdentifier("1", columnProperties, DataTypes.STRING);
-    dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    dictionaryCacheLoader = new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
-    dictionaryInfo = new ColumnDictionaryInfo(DataTypes.STRING);
-    new MockUp<CarbonDictionaryReaderImpl>() {
-      @Mock @SuppressWarnings("unused") Iterator<byte[]> read(long startOffset, long endOffset)
-          throws IOException {
-        ColumnDictionaryChunk columnDictionaryChunk = new ColumnDictionaryChunk();
-        ByteBuffer byteBuffer1 = ByteBuffer.wrap("c".getBytes());
-        ByteBuffer byteBuffer2 = ByteBuffer.wrap("d".getBytes());
-        columnDictionaryChunk.setValues(Arrays.asList(byteBuffer1, byteBuffer2));
-        return new ColumnDictionaryChunkIterator(Arrays.asList(columnDictionaryChunk));
-      }
-    };
-
-    new MockUp<CarbonDictionarySortIndexReaderImpl>() {
-      @Mock @SuppressWarnings("unused") List<Integer> readSortIndex() throws IOException {
-        return Arrays.asList(1, 2);
-      }
-
-      @Mock @SuppressWarnings("unused") List<Integer> readInvertedSortIndex() throws IOException {
-        return Arrays.asList(1, 2);
-      }
-    };
-  }
-
-  @Test public void testToLoad() throws IOException {
-    new MockUp<ColumnDictionaryInfo>() {
-      @Mock @SuppressWarnings("unused") int getSizeOfLastDictionaryChunk() {
-        return 9999;
-      }
-    };
-    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
-    assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 4);
-  }
-
-  @Test public void testToLoadWithsizeOfOneDictionaryChunkLessThanZero() throws IOException {
-    new MockUp<ColumnDictionaryInfo>() {
-      @Mock @SuppressWarnings("unused") int getSizeOfLastDictionaryChunk() {
-        return 10000;
-      }
-    };
-    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
-    assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 2);
-  }
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryChunksWrapperTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryChunksWrapperTest.java
deleted file mode 100644
index 0ae8004..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryChunksWrapperTest.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test class for dictionary chuck wrapper
- */
-public class DictionaryChunksWrapperTest {
-
-  private List<List<byte[]>> dictionaryChuncks;
-  private DictionaryChunksWrapper dictionaryChunksWrapper;
-  private List<byte[]> expectedData;
-
-  /**
-   * init resources
-   *
-   * @throws Exception
-   */
-  @Before public void setUp() throws Exception {
-    dictionaryChuncks = prepareData();
-    expectedData = prepareExpectedData();
-    dictionaryChunksWrapper = new DictionaryChunksWrapper(dictionaryChuncks);
-  }
-
-  /**
-   * The method returns the list<List<byte[]>>
-   *
-   * @return
-   */
-  private List<List<byte[]>> prepareData() {
-    List<List<byte[]>> dictionaryChunks = new ArrayList<>();
-    List<byte[]> chunks = new ArrayList<>();
-    chunks.add("d".getBytes());
-    chunks.add("b".getBytes());
-    chunks.add("c".getBytes());
-    chunks.add("a".getBytes());
-    dictionaryChunks.add(new ArrayList<byte[]>());
-    dictionaryChunks.add(chunks);
-    return dictionaryChunks;
-  }
-
-  private List<byte[]> prepareExpectedData() {
-    List<byte[]> chunks = new ArrayList<>();
-    chunks.add("d".getBytes());
-    chunks.add("b".getBytes());
-    chunks.add("c".getBytes());
-    chunks.add("a".getBytes());
-    return chunks;
-  }
-
-  /**
-   * release resources
-   *
-   * @throws Exception
-   */
-  @After public void tearDown() throws Exception {
-    dictionaryChunksWrapper = null;
-    expectedData = null;
-    dictionaryChuncks = null;
-  }
-
-  /**
-   * The test the next method
-   *
-   * @throws Exception
-   */
-  @Test public void testNext() throws Exception {
-    List<byte[]> actual = new ArrayList<>();
-    while (dictionaryChunksWrapper.hasNext()) {
-      actual.add(dictionaryChunksWrapper.next());
-    }
-    Assert.assertEquals(expectedData.size(), actual.size());
-    for (int i = 0; i < expectedData.size(); i++) {
-      Assert.assertArrayEquals(actual.get(i), expectedData.get(i));
-    }
-  }
-
-  /**
-   * The method validate the size
-   *
-   * @throws Exception
-   */
-  @Test public void getSize() throws Exception {
-    int size = dictionaryChunksWrapper.getSize();
-    Assert.assertEquals(4, size);
-  }
-
-  @Test(expected = UnsupportedOperationException.class) public void testRemove() {
-    dictionaryChunksWrapper.remove();
-  }
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
deleted file mode 100644
index eb55047..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifierTest.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertEquals;
-import static junit.framework.TestCase.assertNotNull;
-import static junit.framework.TestCase.assertTrue;
-
-public class DictionaryColumnUniqueIdentifierTest {
-
-  private static DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier1;
-  private static DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier2;
-  private static DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier3;
-
-  @BeforeClass public static void setUp() throws Exception {
-    CarbonTableIdentifier carbonTableIdentifier1 =
-        new CarbonTableIdentifier("testDatabase", "testTable", "1");
-    CarbonTableIdentifier carbonTableIdentifier2 =
-        new CarbonTableIdentifier("testDatabase", "testTable", "2");
-    AbsoluteTableIdentifier absoluteTableIdentifier1 = AbsoluteTableIdentifier.from("storepath",
-        carbonTableIdentifier1);
-    AbsoluteTableIdentifier absoluteTableIdentifier2 = AbsoluteTableIdentifier.from("storepath",
-        carbonTableIdentifier2);
-    Map<String, String> properties = new HashMap<>();
-    ColumnIdentifier columnIdentifier = new ColumnIdentifier("2", properties, DataTypes.STRING);
-    ColumnIdentifier columnIdentifier2 = new ColumnIdentifier("1", properties, DataTypes.INT);
-    dictionaryColumnUniqueIdentifier1 =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier1, columnIdentifier,
-            DataTypes.STRING, null);
-    dictionaryColumnUniqueIdentifier2 =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier2, columnIdentifier2,
-            DataTypes.STRING, null);
-    dictionaryColumnUniqueIdentifier3 =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier2, columnIdentifier,
-            DataTypes.STRING, null);
-  }
-
-  @Test public void testToGetDataType() {
-    assertEquals(dictionaryColumnUniqueIdentifier1.getDataType(), DataTypes.STRING);
-  }
-
-  @Test public void testForEqualsWithDifferentObjectsWithDifferentColumnIdentifier() {
-    assertTrue(!dictionaryColumnUniqueIdentifier1.equals(dictionaryColumnUniqueIdentifier2));
-  }
-
-  @Test public void testForEqualsWithDifferentObjectsWithSameCarbonTableIdentifier() {
-    assertTrue(!dictionaryColumnUniqueIdentifier3.equals(dictionaryColumnUniqueIdentifier2));
-  }
-
-  @Test public void testForEquals() {
-    assertTrue(dictionaryColumnUniqueIdentifier1.equals(dictionaryColumnUniqueIdentifier1));
-  }
-
-  @Test public void testForEqualsWithNull() {
-    assertNotNull(dictionaryColumnUniqueIdentifier1);
-  }
-
-  @Test public void testForEqualsWithDifferentClass() {
-    assertTrue(!dictionaryColumnUniqueIdentifier1.equals(""));
-  }
-
-  @Test public void testToGetHashCode() {
-    new MockUp<CarbonTableIdentifier>() {
-      @SuppressWarnings("unused") @Mock public int hashCode() {
-        return 1;
-      }
-    };
-    new MockUp<ColumnIdentifier>() {
-      @SuppressWarnings("unused") @Mock public int hashCode() {
-        return 2;
-      }
-    };
-    assertEquals(dictionaryColumnUniqueIdentifier1.hashCode(), 937100380);
-  }
-}
\ No newline at end of file
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
deleted file mode 100644
index c36c89d..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
+++ /dev/null
@@ -1,285 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test class to test the functionality of forward dictionary cache
- */
-public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
-
-  private Cache forwardDictionaryCache;
-
-  @Before public void setUp() throws Exception {
-    init();
-    this.databaseName = props.getProperty("database", "testSchema");
-    this.tableName = props.getProperty("tableName", "carbon");
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    carbonTableIdentifier =
-        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    identifier =
-        AbsoluteTableIdentifier.from(carbonStorePath + "/" + databaseName + "/" + tableName,
-            carbonTableIdentifier);
-    columnIdentifiers = new String[] { "name", "place" };
-    deleteStorePath();
-    prepareDataSet();
-    createDictionaryCacheObject();
-  }
-
-  @After public void tearDown() throws Exception {
-    carbonTableIdentifier = null;
-    identifier = null;
-    forwardDictionaryCache = null;
-    deleteStorePath();
-  }
-
-  private void createDictionaryCacheObject() {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    forwardDictionaryCache =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
-  }
-
-  @Test public void get() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // write sort index file
-    writeSortIndexFile(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the forward dictionary object
-    Dictionary forwardDictionary =
-        (Dictionary) forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // forward dictionary object should not be null
-    assertTrue(null != forwardDictionary);
-    // compare that surrogate key for data inserted and actual data should be same
-    compareSurrogateKeyData(dataSet1, forwardDictionary);
-    // decrement its access count
-    forwardDictionary.clear();
-    // remove keys from lru cache
-    removeKeyFromLRUCache(forwardDictionaryCache);
-  }
-
-  @Test public void getAll() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // write sort index file
-    writeSortIndexFile(dataSet1, columnIdentifier);
-    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers = new ArrayList<>(3);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-    // prepare dictionary writer and write data
-    columnIdentifier = columnIdentifiers[1];
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // write sort index file
-    writeSortIndexFile(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    dictionaryColumnUniqueIdentifier = createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-    // get the forward dictionary object
-    List<Dictionary> forwardDictionaryList =
-        (List<Dictionary>) forwardDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
-    for (Dictionary forwardDictionary : forwardDictionaryList) {
-      // forward dictionary object should not be null
-      assertTrue(null != forwardDictionary);
-      // compare that surrogate key for data inserted and actual data should be same
-      compareSurrogateKeyData(dataSet1, forwardDictionary);
-      // decrement its access count
-      forwardDictionary.clear();
-    }
-    // remove keys from lru cache
-    removeKeyFromLRUCache(forwardDictionaryCache);
-  }
-
-  @Test public void testMultipleDictionaryChunks() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // write sort index file
-    writeSortIndexFile(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the forward dictionary object
-    Dictionary forwardDictionary =
-        (Dictionary) forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // forward dictionary object should not be null
-    assertTrue(null != forwardDictionary);
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet2, columnIdentifier);
-    // write sort index file
-    List<String> allDictionaryChunkList = new ArrayList<>(6);
-    allDictionaryChunkList.addAll(dataSet1);
-    allDictionaryChunkList.addAll(dataSet2);
-    writeSortIndexFile(allDictionaryChunkList, columnIdentifier);
-    // get the forward dictionary object
-    forwardDictionary = (Dictionary) forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // forward dictionary object should not be null
-    assertTrue(null != forwardDictionary);
-    // prepare expected result
-    List<String> expected = new ArrayList<>(2);
-    expected.addAll(dataSet1);
-    expected.addAll(dataSet2);
-    // compare the data
-    compareSurrogateKeyData(expected, forwardDictionary);
-    // decrement access count
-    forwardDictionary.clear();
-    // remove keys from lru cache
-    removeKeyFromLRUCache(forwardDictionaryCache);
-  }
-
-  @Test public void testSortedAndInvertedSortIndex() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet3, columnIdentifier);
-    // write sort index file
-    writeSortIndexFile(dataSet3, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the forward dictionary object
-    Dictionary forwardDictionary =
-        (Dictionary) forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // forward dictionary object should not be null
-    assertTrue(null != forwardDictionary);
-    // compare that surrogate key for data inserted and actual data should be same
-    compareSurrogateKeyData(dataSet3, forwardDictionary);
-    // compare the surrogate keys for given dictionary values
-    compareDictionaryValueFromSortedIndex(dataSet3, forwardDictionary);
-    // decrement its access count
-    forwardDictionary.clear();
-    // remove keys from lru cache
-    removeKeyFromLRUCache(forwardDictionaryCache);
-  }
-
-  /**
-   * This method will prepare the sort index data from the given data and write
-   * it to a sort index file
-   *
-   * @param data
-   * @param columnId
-   * @throws IOException
-   */
-  private void writeSortIndexFile(List<String> data, String columnId) throws IOException {
-	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    Map<String, Integer> dataToSurrogateKeyMap = new HashMap<>(data.size());
-    int surrogateKey = 0;
-    List<Integer> invertedIndexList = new ArrayList<>(data.size());
-    for (int i = 0; i < data.size(); i++) {
-      dataToSurrogateKeyMap.put(data.get(i), ++surrogateKey);
-    }
-    List<String> sortedKeyList = new ArrayList<>(dataToSurrogateKeyMap.keySet());
-    Collections.sort(sortedKeyList);
-    List<Integer> sortedIndexList = new ArrayList<>(data.size());
-    int[] invertedIndexArray = new int[sortedKeyList.size()];
-    for (int i = 0; i < sortedKeyList.size(); i++) {
-      Integer key = dataToSurrogateKeyMap.get(sortedKeyList.get(i));
-      sortedIndexList.add(key);
-      invertedIndexArray[--key] = i + 1;
-    }
-    for (int i = 0; i < invertedIndexArray.length; i++) {
-      invertedIndexList.add(invertedIndexArray[i]);
-    }
-    CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
-    try {
-      dictionarySortIndexWriter.writeSortIndex(sortedIndexList);
-      dictionarySortIndexWriter.writeInvertedSortIndex(invertedIndexList);
-    } finally {
-      dictionarySortIndexWriter.close();
-    }
-  }
-
-  /**
-   * This method will compare the actual data with expected data
-   *
-   * @param data
-   * @param forwardDictionary
-   */
-  private void compareSurrogateKeyData(List<String> data, Dictionary forwardDictionary) {
-    int surrogateKey = 0;
-    for (int i = 0; i < data.size(); i++) {
-      surrogateKey++;
-      String dictionaryValue = forwardDictionary.getDictionaryValueForKey(surrogateKey);
-      assertTrue(data.get(i).equals(dictionaryValue));
-    }
-  }
-
-  /**
-   * This method will get the dictionary value from sorted index and compare with the data set
-   *
-   * @param data
-   * @param forwardDictionary
-   */
-  private void compareDictionaryValueFromSortedIndex(List<String> data,
-      Dictionary forwardDictionary) {
-    int expectedSurrogateKey = 0;
-    for (int i = 0; i < data.size(); i++) {
-      expectedSurrogateKey++;
-      String expectedDictionaryValue = data.get(i);
-      int actualSurrogateKey = forwardDictionary.getSurrogateKey(expectedDictionaryValue);
-      assertTrue(actualSurrogateKey == expectedSurrogateKey);
-      int sortedIndex = forwardDictionary.getSortedIndex(actualSurrogateKey);
-      String actualDictionaryValue =
-          forwardDictionary.getDictionaryValueFromSortedIndex(sortedIndex);
-      assertTrue(expectedDictionaryValue.equals(actualDictionaryValue));
-    }
-  }
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java
deleted file mode 100644
index aee6b9e..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryTest.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import mockit.Mock;
-import mockit.MockUp;
-
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static junit.framework.TestCase.*;
-
-public class ForwardDictionaryTest {
-
-  private static ForwardDictionary forwardDictionary;
-
-  @BeforeClass public static void setUp() {
-    ColumnDictionaryInfo columnDictionaryInfo = new ColumnDictionaryInfo(DataTypes.INT);
-    forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
-  }
-
-  @Test public void testToGetSurrogateKeyForStringInput() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @Mock @SuppressWarnings("unused") public int getSurrogateKey(String value) {
-        return 123;
-      }
-    };
-    int expectedResult = 123;
-    assertEquals(forwardDictionary.getSurrogateKey("123"), expectedResult);
-  }
-
-  @Test public void testToGetSurrogateKeyForByteInput() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @Mock @SuppressWarnings("unused") public int getSurrogateKey(byte[] value) {
-        return 123;
-      }
-    };
-    int expectedResult = 123;
-    assertEquals(forwardDictionary.getSurrogateKey("123".getBytes()), expectedResult);
-  }
-
-  @Test public void testToGetDictionaryValueForKey() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @Mock @SuppressWarnings("unused") public String getDictionaryValueForKey(int surrogateKey) {
-        System.out.print("Mocked");
-        return "123";
-      }
-    };
-    String expectedResult = "123";
-    assertEquals(forwardDictionary.getDictionaryValueForKey(123), expectedResult);
-  }
-
-  @Test public void testToGetSortedIndex() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock public int getSortedIndex(int surrogateKey) {
-        System.out.print("Mocked");
-        return 1;
-      }
-    };
-    int expectedResult = 1;
-    int sortedIndex = forwardDictionary.getSortedIndex(123);
-    assertEquals(sortedIndex, expectedResult);
-  }
-
-  @Test public void testToGetDictionaryValueFromSortedIndex() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock
-      public String getDictionaryValueFromSortedIndex(int sortedIndex) {
-        System.out.print("Mocked");
-        return "A";
-      }
-    };
-    String expectedResult = "A";
-    String dictionaryValue = forwardDictionary.getDictionaryValueFromSortedIndex(123);
-    assertEquals(dictionaryValue, expectedResult);
-  }
-
-  @Test public void testToGetDictionaryChunks() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock public DictionaryChunksWrapper getDictionaryChunks() {
-        System.out.print("Mocked");
-        List<List<byte[]>> dictionaryChunks =
-            Arrays.asList(Arrays.asList("123".getBytes()), Arrays.asList("321".getBytes()));
-        return new DictionaryChunksWrapper(dictionaryChunks);
-      }
-    };
-    DictionaryChunksWrapper dictionaryValue = forwardDictionary.getDictionaryChunks();
-    int expectedResult = 2;
-    assertEquals(dictionaryValue.getSize(), expectedResult);
-  }
-
-  @Test public void testToGtSurrogateKeyByIncrementalSearch() {
-    new MockUp<ColumnDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock
-      public void getIncrementalSurrogateKeyFromDictionary(List<byte[]> byteValuesOfFilterMembers,
-          List<Integer> surrogates) {
-        surrogates.add(1);
-      }
-    };
-    List<String> evaluateResultList = Arrays.asList("1", "2");
-    List<Integer> surrogates = new ArrayList<>(1);
-    forwardDictionary.getSurrogateKeyByIncrementalSearch(evaluateResultList, surrogates);
-    Integer expectedResult = 1;
-    assertEquals(surrogates.get(0), expectedResult);
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
deleted file mode 100644
index d2bf2e3..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-
-import mockit.Mock;
-import mockit.MockUp;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test class to test the functionality of reverse dictionary cache
- */
-public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
-
-  protected Cache reverseDictionaryCache;
-
-  @Before public void setUp() throws Exception {
-    init();
-    this.databaseName = props.getProperty("database", "testSchema");
-    this.tableName = props.getProperty("tableName", "carbon");
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    carbonTableIdentifier =
-        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    identifier = AbsoluteTableIdentifier.from(
-        carbonStorePath + "/" + databaseName + "/" + tableName, carbonTableIdentifier);
-    columnIdentifiers = new String[] { "name", "place" };
-    deleteStorePath();
-    prepareDataSet();
-    createDictionaryCacheObject();
-  }
-
-  @After public void tearDown() throws Exception {
-    carbonTableIdentifier = null;
-    reverseDictionaryCache = null;
-    identifier = null;
-    deleteStorePath();
-  }
-
-  private void createDictionaryCacheObject() {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    cacheProvider.dropAllCache();
-    reverseDictionaryCache =
-        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
-  }
-
-  @Test public void get() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the reverse dictionary object
-    Dictionary reverseDictionary =
-        (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // reverse dictionary object should not be null
-    assertTrue(null != reverseDictionary);
-    // compare that surrogate key for data inserted and actual data should be same
-    compareSurrogateKeyData(dataSet1, reverseDictionary);
-    // decrement its access count
-    reverseDictionary.clear();
-    // remove keys from lru cache
-    removeKeyFromLRUCache(reverseDictionaryCache);
-  }
-
-  @Test public void getAll() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers = new ArrayList<>(3);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-    // prepare dictionary writer and write data
-    columnIdentifier = columnIdentifiers[1];
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    dictionaryColumnUniqueIdentifier = createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-    // get the reverse dictionary object
-    List<Dictionary> reverseDictionaryList =
-        (List<Dictionary>) reverseDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
-    for (Dictionary reverseDictionary : reverseDictionaryList) {
-      // reverse dictionary object should not be null
-      assertTrue(null != reverseDictionary);
-      // compare that surrogate key for data inserted and actual data should be same
-      compareSurrogateKeyData(dataSet1, reverseDictionary);
-      // decrement its access count
-      reverseDictionary.clear();
-    }
-    // remove keys from lru cache
-    removeKeyFromLRUCache(reverseDictionaryCache);
-  }
-
-  @Test public void getIfPresent() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the reverse dictionary object
-    Dictionary reverseDictionary =
-        (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // reverse dictionary object should not be null
-    assertTrue(null != reverseDictionary);
-    reverseDictionary =
-        (Dictionary) reverseDictionaryCache.getIfPresent(dictionaryColumnUniqueIdentifier);
-    // compare that surrogate key for data inserted and actual data should be same
-    compareSurrogateKeyData(dataSet1, reverseDictionary);
-    // remove the identifier from lru cache
-    reverseDictionaryCache.invalidate(dictionaryColumnUniqueIdentifier);
-    // use getIfPresent API to get the reverse dictionary cache again
-    reverseDictionary =
-        (Dictionary) reverseDictionaryCache.getIfPresent(dictionaryColumnUniqueIdentifier);
-    // as key has been removed from lru cache object should not be found
-    assertTrue(null == reverseDictionary);
-  }
-
-  @Test public void testLRUCacheForMaxSize() throws Exception {
-    // delete store path
-    deleteStorePath();
-    // mock get end offset method so that required size is greater than
-    // available size limit
-    new MockUp<CarbonDictionaryColumnMetaChunk>() {
-      @Mock public long getEnd_offset() {
-        return 123456789L;
-      }
-    };
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the reverse dictionary object
-    Dictionary reverseDictionary = null;
-    try {
-      reverseDictionary = (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-      fail("not throwing exception");
-    } catch (Exception e) {
-      assertTrue(e instanceof IOException);
-    }
-    assertEquals(null, reverseDictionary);
-  }
-
-  @Test public void testLRUCacheForKeyDeletionAfterMaxSizeIsReached() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the reverse dictionary object
-    Dictionary reverseDictionary =
-        (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // reverse dictionary object should not be null
-    assertTrue(null != reverseDictionary);
-    // decrement access count
-    reverseDictionary.clear();
-    // mock get end offset method so that required size is greater than
-    // available size limit
-    new MockUp<CarbonDictionaryColumnMetaChunk>() {
-      @Mock public long getEnd_offset() {
-        return 10445000L;
-      }
-    };
-    columnIdentifier = columnIdentifiers[1];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    dictionaryColumnUniqueIdentifier = createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the reverse dictionary object
-    // lru cache should delete the existing key and empty the size for new key addition
-    reverseDictionary = (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // reverse dictionary object should not be null
-    assertTrue(null != reverseDictionary);
-    // remove keys from lru cache
-    removeKeyFromLRUCache(reverseDictionaryCache);
-  }
-
-  @Test public void testMultipleDictionaryChunks() throws Exception {
-    // delete store path
-    deleteStorePath();
-    String columnIdentifier = columnIdentifiers[0];
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet1, columnIdentifier);
-    // create dictionary column unique identifier instance
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        createDictionaryColumnUniqueIdentifier(columnIdentifier);
-    // get the reverse dictionary object
-    Dictionary reverseDictionary =
-        (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // reverse dictionary object should not be null
-    assertTrue(null != reverseDictionary);
-    // prepare dictionary writer and write data
-    prepareWriterAndWriteData(dataSet2, columnIdentifier);
-    // get the reverse dictionary object
-    reverseDictionary = (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    // reverse dictionary object should not be null
-    assertTrue(null != reverseDictionary);
-    // prepare expected result
-    List<String> expected = new ArrayList<>(2);
-    expected.addAll(dataSet1);
-    expected.addAll(dataSet2);
-    // compare the data
-    compareSurrogateKeyData(expected, reverseDictionary);
-    // decrement access count
-    reverseDictionary.clear();
-    // remove keys from lru cache
-    removeKeyFromLRUCache(reverseDictionaryCache);
-  }
-
-  /**
-   * This method will compare the actual data with expected data
-   *
-   * @param data
-   * @param reverseDictionary
-   */
-  private void compareSurrogateKeyData(List<String> data, Dictionary reverseDictionary) {
-    int surrogateKey = 0;
-    for (int i = 0; i < data.size(); i++) {
-      surrogateKey++;
-      assertTrue(surrogateKey == reverseDictionary.getSurrogateKey(data.get(i)));
-    }
-  }
-  protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
-	      String columnId) {
-	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.DOUBLE);
-    return new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier);
-	  }
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryTest.java
deleted file mode 100644
index 6c3ba7f..0000000
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryTest.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.cache.dictionary;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.List;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class ReverseDictionaryTest {
-
-  private static ReverseDictionary reverseDictionary;
-
-  @BeforeClass public static void setUp() throws Exception {
-    ColumnReverseDictionaryInfo columnReverseDictionaryInfo = new ColumnReverseDictionaryInfo();
-    reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
-  }
-
-  @Test public void testToGetSurrogateKey() {
-    new MockUp<ColumnReverseDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock public int getSurrogateKey(byte[] value) {
-        return 123;
-      }
-    };
-    int surrogateKey = reverseDictionary.getSurrogateKey("123".getBytes());
-    int expectedResult = 123;
-    assertEquals(surrogateKey, expectedResult);
-  }
-
-  @Test public void testToGetDictionaryValueForKey() {
-    new MockUp<ColumnReverseDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock public String getDictionaryValueForKey(int surrogateKey) {
-        return "123";
-      }
-    };
-    String dictionaryValue = reverseDictionary.getDictionaryValueForKey(123);
-    String expectedResult = "123";
-    assertEquals(dictionaryValue, expectedResult);
-  }
-
-  @Test public void testToGetSortedIndex() {
-    new MockUp<ColumnReverseDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock public int getSortedIndex(int surrogateKey) {
-        return 1;
-      }
-    };
-    int sortedIndex = reverseDictionary.getSortedIndex(123);
-    int expectedResult = 1;
-    assertEquals(sortedIndex, expectedResult);
-  }
-
-  @Test public void testToGetDictionaryValueFromSortedIndex() {
-    new MockUp<ColumnReverseDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock
-      public String getDictionaryValueFromSortedIndex(int sortedIndex) {
-        return "A";
-      }
-    };
-    String dictionaryValue = reverseDictionary.getDictionaryValueFromSortedIndex(123);
-    String expectedResult = "A";
-    assertEquals(dictionaryValue, expectedResult);
-  }
-
-  @Test public void testToGetDictionaryChunks() {
-    new MockUp<ColumnReverseDictionaryInfo>() {
-      @SuppressWarnings("unused") @Mock public DictionaryChunksWrapper getDictionaryChunks() {
-        List<List<byte[]>> dictionaryChunks =
-            Arrays.asList(Arrays.asList("123".getBytes()), Arrays.asList("321".getBytes()));
-        return new DictionaryChunksWrapper(dictionaryChunks);
-      }
-    };
-    DictionaryChunksWrapper dictionaryValue = reverseDictionary.getDictionaryChunks();
-    int expectedResult = 2;
-    assertEquals(dictionaryValue.getSize(), expectedResult);
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
deleted file mode 100644
index edefd09..0000000
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.client;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-import org.apache.carbondata.core.dictionary.server.DictionaryServer;
-import org.apache.carbondata.core.dictionary.server.NonSecureDictionaryServer;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-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.TableInfo;
-import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Test class to test dictionary client functionality.
- */
-public class DictionaryClientTest {
-
-  private static ColumnSchema empColumnSchema;
-  private static CarbonDimension empDimension;
-  private static ColumnSchema ageColumnSchema;
-  private static CarbonDimension ageDimension;
-  private static TableSchema tableSchema;
-  private static TableInfo tableInfo;
-  private static String storePath;
-  private static DictionaryServer server;
-  private static String host;
-
-  @BeforeClass public static void setUp() throws Exception {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-
-    // Create two column schemas and dimensions for the table
-    empColumnSchema = new ColumnSchema();
-    empColumnSchema.setColumnName("empNameCol");
-    empColumnSchema.setColumnUniqueId("empNameCol");
-    empColumnSchema.setDimensionColumn(true);
-    empColumnSchema.setDataType(DataTypes.STRING);
-    empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0);
-
-    ageColumnSchema = new ColumnSchema();
-    ageColumnSchema.setColumnName("ageNameCol");
-    ageColumnSchema.setDataType(DataTypes.SHORT_INT);
-    ageColumnSchema.setColumnUniqueId("ageNameCol");
-    ageColumnSchema.setDimensionColumn(true);
-    ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0);
-
-    // Create a Table
-    tableSchema = new TableSchema();
-    tableSchema.setTableName("TestTable");
-    tableSchema.setTableId("1");
-    tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-
-    tableInfo = new TableInfo();
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setTableUniqueName("TestTable");
-    tableInfo.setDatabaseName("test");
-    storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setTablePath(storePath + "/" + "test" + "/" + "TestTable");
-    CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-
-    // Add the created table to metadata
-    metadata.addCarbonTable(carbonTable);
-
-    // Start the server for testing the client
-    server = NonSecureDictionaryServer.getInstance(5678, carbonTable);
-    host = server.getHost();
-  }
-
-  @Test public void testClient() throws Exception {
-    NonSecureDictionaryClient client = new NonSecureDictionaryClient();
-    client.startClient(null, host, 5678, false);
-
-    Thread.sleep(1000);
-    // Create a dictionary key
-    DictionaryMessage empKey = new DictionaryMessage();
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("FirstKey");
-
-    // Test dictionary initialization call
-    int count = 2;
-    // Test dictionary generation
-    for (; count <= 10000; count++) {
-      empKey.setType(DictionaryMessageType.DICT_GENERATION);
-      empKey.setTableUniqueId("1");
-      empKey.setData("FirstKey" + count);
-      DictionaryMessage val = client.getDictionary(empKey);
-      Assert.assertEquals(count, val.getDictionaryValue());
-    }
-
-    // Test dictionary generation with big messages
-    for (; count <= 10010; count++) {
-      empKey.setType(DictionaryMessageType.DICT_GENERATION);
-      empKey.setData(
-          "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + "FirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKeyFirstKey"
-              + count);
-      DictionaryMessage val = client.getDictionary(empKey);
-      Assert.assertEquals(count, val.getDictionaryValue());
-    }
-    // Test size function
-    empKey.setType(DictionaryMessageType.SIZE);
-    DictionaryMessage val = client.getDictionary(empKey);
-    Assert.assertEquals(10010, val.getDictionaryValue());
-
-
-    client.shutDown();
-
-    // Shutdown the server
-  }
-
-  @Test public void testToCheckIfCorrectTimeOutExceptionMessageIsThrown() {
-    new MockUp<LinkedBlockingQueue<DictionaryMessage>>() {
-      @SuppressWarnings("unused")
-      @Mock
-      DictionaryMessage poll(long timeout, TimeUnit unit) throws InterruptedException {
-        return null;
-      }
-    };
-    try {
-      testClient();
-      Assert.fail();
-    } catch (Exception e) {
-      Assert.assertFalse(e.getMessage().contains("data"));
-    }
-  }
-
-  @After public void tearDown() {
-    // Cleanup created files
-    CarbonMetadata.getInstance().removeTable(tableInfo.getTableUniqueName());
-    cleanUpDirectory(new File(storePath));
-  }
-
-  private static void cleanUpDirectory(File path) {
-    File[] files = path.listFiles();
-    if (null == files) {
-      return;
-    }
-    for (File file : files) {
-      if (file.isDirectory()) cleanUpDirectory(file);
-      else file.delete();
-    }
-    path.delete();
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
deleted file mode 100644
index ef06ac1..0000000
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.io.File;
-import java.util.Arrays;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.TableInfo;
-import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test class to test incremental column dictionary generator functionality
- */
-public class IncrementalColumnDictionaryGeneratorTest {
-
-  private CarbonTable carbonTable;
-  private CarbonDimension carbonDimension;
-
-  @Before public void setUp() throws Exception {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-    ColumnSchema columnSchema = new ColumnSchema();
-    columnSchema.setColumnName("empName");
-    columnSchema.setDataType(DataTypes.STRING);
-    TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName("TestTable");
-    tableSchema.setListOfColumns(Arrays.asList(columnSchema));
-    TableInfo tableInfo = new TableInfo();
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setTableUniqueName("TestTable");
-    tableInfo.setDatabaseName("test");
-    String storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
-    carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-    carbonDimension = new CarbonDimension(columnSchema,0,0,0,0);
-  }
-
-  @Test public void generateKeyOnce() throws Exception {
-    // Create the generator and add the key to dictionary
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-    Integer key = generator.generateKey("First");
-    assertEquals(new Integer(11), key);
-  }
-
-  @Test public void generateKeyTwice() throws Exception {
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-    Integer key = generator.generateKey("First");
-
-    // Add one more key and check if it works fine.
-    key = generator.generateKey("Second");
-    assertEquals(new Integer(12), key);
-  }
-
-  @Test public void generateKeyAgain() throws Exception {
-    // Create the generator and add the key to dictionary
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-    Integer key = generator.generateKey("First");
-
-    // Add the same key again anc check if the value is correct
-    key = generator.generateKey("First");
-    assertEquals(new Integer(11), key);
-  }
-
-  @Test public void getKey() throws Exception {
-    // Create the generator and add the key to dictionary
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-    Integer generatedKey = generator.generateKey("First");
-
-    // Get the value of the key from dictionary and check if it matches with the created value
-    Integer obtainedKey = generator.getKey("First");
-    assertEquals(generatedKey, obtainedKey);
-  }
-
-  @Test public void getKeyInvalid() throws Exception {
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-
-    // Try to get value for an invalid key
-    Integer obtainedKey = generator.getKey("Second");
-    assertNull(obtainedKey);
-  }
-
-  @Test public void getOrGenerateKey() throws Exception {
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-
-    // Test first with generating a key and then trying geOrGenerate
-    Integer generatedKey = generator.generateKey("First");
-    Integer obtainedKey = generator.getOrGenerateKey("First");
-    assertEquals(generatedKey, obtainedKey);
-
-    // Test directly with getOrGenerate for another key
-    obtainedKey = generator.getOrGenerateKey("Second");
-    assertEquals(new Integer(12), obtainedKey);
-
-  }
-
-  @Test public void writeDictionaryData() throws Exception {
-    //Create required column schema
-    ColumnSchema columnSchema = new ColumnSchema();
-    columnSchema.setColumnName("empNameCol");
-    columnSchema.setDataType(DataTypes.STRING);
-    columnSchema.setColumnUniqueId("empNameCol");
-    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0);
-
-    // Create the generator and add the keys to dictionary
-    IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
-
-    // Create a table schema for saving the dictionary
-    TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName("TestTable");
-    tableSchema.setListOfColumns(Arrays.asList(columnSchema));
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-    TableInfo tableInfo = new TableInfo();
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setTableUniqueName("TestTable");
-    tableInfo.setDatabaseName("test");
-
-    String storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    File dictPath = new File(storePath + "/test/TestTable/Metadata/");
-    System.out.print(dictPath.mkdirs());
-
-    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
-    CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-
-    // Add the table to metadata
-    metadata.addCarbonTable(carbonTable);
-
-    /// Write the dictionary and verify whether its written successfully
-    generator.writeDictionaryData();
-    File dictionaryFile = new File(dictPath, "empNameCol.dict");
-    System.out.println(dictionaryFile.getCanonicalPath());
-    assertTrue(dictionaryFile.exists());
-    dictionaryFile.delete();
-
-    // cleanup created files
-    metadata.removeTable(carbonTable.getTableUniqueName());
-    cleanUpDirectory(new File(storePath));
-
-  }
-
-  private void cleanUpDirectory(File path) {
-    File[] files = path.listFiles();
-    if (null == files) {
-      return;
-    }
-    for (File file : files) {
-      if (file.isDirectory()) cleanUpDirectory(file);
-      else file.delete();
-    }
-    path.delete();
-  }
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
deleted file mode 100644
index 5ff51cc..0000000
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.io.File;
-import java.util.Arrays;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-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.TableInfo;
-import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import org.junit.After;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test class to test server column dictionary generator functionality
- */
-public class ServerDictionaryGeneratorTest {
-
-  private static ColumnSchema empColumnSchema;
-  private static CarbonDimension empDimension;
-  private static ColumnSchema ageColumnSchema;
-  private static CarbonDimension ageDimension;
-  private static TableSchema tableSchema;
-  private static TableInfo tableInfo;
-  private static String storePath;
-  private static CarbonTable carbonTable;
-
-  @BeforeClass public static void setUp() throws Exception {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-
-    // Create two column schemas and dimensions for the table
-    empColumnSchema = new ColumnSchema();
-    empColumnSchema.setColumnName("empNameCol");
-    empColumnSchema.setDataType(DataTypes.STRING);
-    empColumnSchema.setColumnUniqueId("empNameCol");
-    empColumnSchema.setDimensionColumn(true);
-    empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0,0);
-
-    ageColumnSchema = new ColumnSchema();
-    ageColumnSchema.setColumnName("empNameCol");
-    ageColumnSchema.setColumnUniqueId("empNameCol");
-    ageColumnSchema.setDataType(DataTypes.SHORT_INT);
-    ageColumnSchema.setDimensionColumn(true);
-    ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0);
-
-    // Create a Table
-    tableSchema = new TableSchema();
-    tableSchema.setTableName("TestTable");
-    tableSchema.setTableId("1");
-    tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-
-    tableInfo = new TableInfo();
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setTableUniqueName("TestTable");
-    tableInfo.setDatabaseName("test");
-    storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
-    carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-    // Add the created table to metadata
-    metadata.addCarbonTable(carbonTable);
-  }
-
-  @Test public void generateKeyOnce() throws Exception {
-    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
-    // Generate dictionary for one key
-    DictionaryMessage empKey = new DictionaryMessage();
-    empKey.setTableUniqueId("1");
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
-    Integer value = serverDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(2), value);
-  }
-
-  @Test public void generateKeyTwice() throws Exception {
-    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
-
-    // Generate dictionary for same key twice
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setTableUniqueId("1");
-    firstKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
-    Integer value = serverDictionaryGenerator.generateKey(firstKey);
-    assertEquals(new Integer(2), value);
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setColumnName(empColumnSchema.getColumnName());
-    secondKey.setData("FirstKey");
-    secondKey.setTableUniqueId("1");
-    value = serverDictionaryGenerator.generateKey(secondKey);
-    assertEquals(new Integer(2), value);
-  }
-
-  @Test public void generateKeyAgain() throws Exception {
-    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
-    // Generate dictionary for two different keys
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setTableUniqueId("1");
-    firstKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
-    Integer value = serverDictionaryGenerator.generateKey(firstKey);
-    assertEquals(new Integer(2), value);
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setTableUniqueId("1");
-    secondKey.setColumnName(empColumnSchema.getColumnName());
-    secondKey.setData("SecondKey");
-    value = serverDictionaryGenerator.generateKey(secondKey);
-    assertEquals(new Integer(3), value);
-  }
-
-  @Test public void size() throws Exception {
-    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
-    //Add keys for first Column
-    DictionaryMessage empKey = new DictionaryMessage();
-    //Add key 1
-    empKey.setData("FirstKey");
-    empKey.setTableUniqueId("1");
-    empKey.setColumnName(ageColumnSchema.getColumnName());
-    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
-    serverDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(2), serverDictionaryGenerator.size(empKey));
-
-    //Add key 2
-    empKey = new DictionaryMessage();
-    empKey.setTableUniqueId("1");
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("SecondKey");
-    serverDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(3), serverDictionaryGenerator.size(empKey));
-
-    //Add key 3
-    empKey = new DictionaryMessage();
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setTableUniqueId("1");
-    empKey.setData("ThirdKey");
-    serverDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(4), serverDictionaryGenerator.size(empKey));
-  }
-
-  @Test public void writeDictionaryData() throws Exception {
-    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setTableUniqueId("1");
-    firstKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
-
-    //Update generator with a new dimension
-
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setColumnName(ageColumnSchema.getColumnName());
-    secondKey.setTableUniqueId("1");
-    secondKey.setData("SecondKey");
-    serverDictionaryGenerator.generateKey(secondKey);
-    File dictPath = new File(storePath + "/test/TestTable/Metadata/");
-    dictPath.mkdirs();
-    serverDictionaryGenerator.writeTableDictionaryData("1");
-
-    File empDictionaryFile = new File(dictPath, empColumnSchema.getColumnName() + ".dict");
-    assertTrue(empDictionaryFile.exists());
-
-    File ageDictionaryFile = new File(dictPath, ageColumnSchema.getColumnName() + ".dict");
-    assertTrue(ageDictionaryFile.exists());
-  }
-
-  @After public void tearDown() {
-    cleanUpDirectory(new File(storePath));
-  }
-
-  private static void cleanUpDirectory(File path) {
-    File[] files = path.listFiles();
-    if (null == files) {
-      return;
-    }
-    for (File file : files) {
-      if (file.isDirectory()) cleanUpDirectory(file);
-      else file.delete();
-    }
-    path.delete();
-    CarbonMetadata.getInstance().removeTable(tableInfo.getTableUniqueName());
-  }
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
deleted file mode 100644
index 96573bc..0000000
--- a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.dictionary.generator;
-
-import java.io.File;
-import java.util.Arrays;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-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.TableInfo;
-import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test class to test table column dictionary generator functionality
- */
-public class TableDictionaryGeneratorTest {
-
-  private ColumnSchema empColumnSchema;
-  private CarbonDimension empDimension;
-  private ColumnSchema ageColumnSchema;
-  private CarbonDimension ageDimension;
-  private TableSchema tableSchema;
-  private TableInfo tableInfo;
-  private String storePath;
-  private CarbonTable carbonTable;
-
-  @Before public void setUp() throws Exception {
-    // enable lru cache by setting cache size
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
-
-    // Create two column schemas and dimensions for the table
-    empColumnSchema = new ColumnSchema();
-    empColumnSchema.setColumnName("empNameCol");
-    empColumnSchema.setColumnUniqueId("empNameCol");
-    empColumnSchema.setDimensionColumn(true);
-    empColumnSchema.setDataType(DataTypes.STRING);
-    empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0);
-
-    ageColumnSchema = new ColumnSchema();
-    ageColumnSchema.setDataType(DataTypes.SHORT_INT);
-    ageColumnSchema.setColumnName("ageNameCol");
-    ageColumnSchema.setColumnUniqueId("ageNameCol");
-    ageColumnSchema.setDimensionColumn(true);
-    ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
-    ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0);
-
-    // Create a Table
-    tableSchema = new TableSchema();
-    tableSchema.setTableName("TestTable");
-    tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-
-    tableInfo = new TableInfo();
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setTableUniqueName("TestTable");
-    tableInfo.setDatabaseName("test");
-    storePath = System.getProperty("java.io.tmpdir") + "/tmp";
-    tableInfo.setTablePath(storePath + "/test" + "/TestTable");
-    carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-    // Add the created table to metadata
-    metadata.addCarbonTable(carbonTable);
-  }
-
-  @Test public void generateKeyOnce() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
-
-    // Generate dictionary for one key
-    DictionaryMessage empKey = new DictionaryMessage();
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("FirstKey");
-    tableDictionaryGenerator.updateGenerator(empKey);
-    Integer value = tableDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(2), value);
-
-  }
-
-  @Test public void generateKeyTwice() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
-
-    // Generate dictionary for same key twice
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setTableUniqueId("1");
-    firstKey.setData("FirstKey");
-    tableDictionaryGenerator.updateGenerator(firstKey);
-    Integer value = tableDictionaryGenerator.generateKey(firstKey);
-    assertEquals(new Integer(2), value);
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setColumnName(empColumnSchema.getColumnName());
-    secondKey.setTableUniqueId("1");
-    secondKey.setData("FirstKey");
-    value = tableDictionaryGenerator.generateKey(secondKey);
-    assertEquals(new Integer(2), value);
-  }
-
-  @Test public void generateKeyAgain() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
-
-    // Generate dictionary for two different keys
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setData("FirstKey");
-    firstKey.setTableUniqueId("1");
-    tableDictionaryGenerator.updateGenerator(firstKey);
-    Integer value = tableDictionaryGenerator.generateKey(firstKey);
-    assertEquals(new Integer(2), value);
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setColumnName(empColumnSchema.getColumnName());
-    secondKey.setData("SecondKey");
-    secondKey.setTableUniqueId("1");
-    tableDictionaryGenerator.updateGenerator(secondKey);
-    value = tableDictionaryGenerator.generateKey(secondKey);
-    assertEquals(new Integer(3), value);
-  }
-
-  @Test public void updateGenerator() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setData("FirstKey");
-    firstKey.setTableUniqueId("1");
-    tableDictionaryGenerator.updateGenerator(firstKey);
-    Integer value = tableDictionaryGenerator.generateKey(firstKey);
-    assertEquals(new Integer(2), value);
-
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setColumnName(ageColumnSchema.getColumnName());
-    secondKey.setData("SecondKey");
-    secondKey.setTableUniqueId("1");
-    tableDictionaryGenerator.updateGenerator(secondKey);
-    //Update generator with a new dimension
-    value = tableDictionaryGenerator.generateKey(secondKey);
-    assertEquals(new Integer(2), value);
-  }
-
-  @Test public void size() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
-    //Add keys for first Column
-    DictionaryMessage empKey = new DictionaryMessage();
-    //Add key 1
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("FirstKey");
-    empKey.setTableUniqueId("1");
-    tableDictionaryGenerator.updateGenerator(empKey);
-    tableDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(2), tableDictionaryGenerator.size(empKey));
-
-    //Add key 2
-    empKey = new DictionaryMessage();
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("SecondKey");
-    empKey.setTableUniqueId("1");
-    tableDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(3), tableDictionaryGenerator.size(empKey));
-
-    //Add key 3
-    empKey = new DictionaryMessage();
-    empKey.setColumnName(empColumnSchema.getColumnName());
-    empKey.setData("ThirdKey");
-    empKey.setTableUniqueId("1");
-    tableDictionaryGenerator.generateKey(empKey);
-    assertEquals(new Integer(4), tableDictionaryGenerator.size(empKey));
-  }
-
-  @Test public void writeDictionaryData() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
-    DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setColumnName(empColumnSchema.getColumnName());
-    firstKey.setData("FirstKey");
-    firstKey.setTableUniqueId("1");
-    tableDictionaryGenerator.updateGenerator(firstKey);
-    tableDictionaryGenerator.generateKey(firstKey);
-
-    DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setColumnName(ageColumnSchema.getColumnName());
-    secondKey.setData("SecondKey");
-    secondKey.setTableUniqueId("1");
-    //Update generator with a new dimension
-    tableDictionaryGenerator.updateGenerator(secondKey);
-    tableDictionaryGenerator.generateKey(secondKey);
-    File dictPath = new File(storePath + "/test/TestTable/Metadata/");
-    dictPath.mkdirs();
-    tableDictionaryGenerator.writeDictionaryData();
-
-    File empDictionaryFile = new File(dictPath, empColumnSchema.getColumnName() + ".dict");
-    assertTrue(empDictionaryFile.exists());
-
-    File ageDictionaryFile = new File(dictPath, ageColumnSchema.getColumnName() + ".dict");
-    assertTrue(ageDictionaryFile.exists());
-  }
-
-  @After public void tearDown() {
-    CarbonMetadata.getInstance().removeTable(tableInfo.getTableUniqueName());
-    cleanUpDirectory(new File(storePath));
-  }
-
-  private void cleanUpDirectory(File path) {
-    File[] files = path.listFiles();
-    if (null == files) {
-      return;
-    }
-    for (File file : files) {
-      if (file.isDirectory()) cleanUpDirectory(file);
-      else file.delete();
-    }
-    path.delete();
-  }
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
index d278d03..d849194 100644
--- a/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
+++ b/core/src/test/java/org/apache/carbondata/core/indexstore/blockletindex/TestBlockletDataMap.java
@@ -20,7 +20,6 @@
 import java.lang.reflect.Method;
 import java.util.BitSet;
 
-import org.apache.carbondata.core.cache.dictionary.AbstractDictionaryCacheTest;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
@@ -33,7 +32,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
-public class TestBlockletDataMap extends AbstractDictionaryCacheTest {
+public class TestBlockletDataMap {
 
   ImplicitIncludeFilterExecutorImpl implicitIncludeFilterExecutor;
   @Before public void setUp() throws Exception {
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryByteArrayWrapperTest.java b/core/src/test/java/org/apache/carbondata/core/localdictionary/DictionaryByteArrayWrapperTest.java
similarity index 94%
rename from core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryByteArrayWrapperTest.java
rename to core/src/test/java/org/apache/carbondata/core/localdictionary/DictionaryByteArrayWrapperTest.java
index bce5b60..1b10ad3 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryByteArrayWrapperTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/localdictionary/DictionaryByteArrayWrapperTest.java
@@ -15,7 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.core.cache.dictionary;
+package org.apache.carbondata.core.localdictionary;
+
+import org.apache.carbondata.core.cache.dictionary.DictionaryByteArrayWrapper;
 
 import net.jpountz.xxhash.XXHashFactory;
 import org.junit.Assert;
@@ -34,7 +36,6 @@
     dictionaryByteArrayWrapper = new DictionaryByteArrayWrapper(data);
     dictionaryByteArrayWrapper1 =
         new DictionaryByteArrayWrapper(data, XXHashFactory.fastestInstance().hash32());
-
   }
 
   @Test public void equalsTestWithSameObject() {
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
deleted file mode 100644
index ecabfd4..0000000
--- a/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.reader;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.service.CarbonCommonFactory;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.apache.thrift.TBase;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-
-public class CarbonDictionaryReaderImplTest {
-  private static AbsoluteTableIdentifier absoluteTableIdentifier = null;
-  private static ColumnIdentifier columnIdentifier = null;
-
-  private static CarbonDictionaryReaderImpl carbonDictionaryReaderImpl = null;
-
-  @BeforeClass public static void setUp() throws Exception {
-    columnIdentifier = new ColumnIdentifier("1", null, null);
-    absoluteTableIdentifier =
-        AbsoluteTableIdentifier.from("tablePath",
-            new CarbonTableIdentifier("dbName", "tableName", UUID.randomUUID().toString()));
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    carbonDictionaryReaderImpl =
-        new CarbonDictionaryReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  @Test public void testRead() throws Exception {
-    new MockUp<CarbonDictionaryMetadataReaderImpl>() {
-      @Mock public List<CarbonDictionaryColumnMetaChunk> read() throws IOException {
-        CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunks =
-            new CarbonDictionaryColumnMetaChunk(1, 8, 60L, 120L, 20);
-        List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunksList =
-            new ArrayList<CarbonDictionaryColumnMetaChunk>();
-        carbonDictionaryColumnMetaChunksList.add(carbonDictionaryColumnMetaChunks);
-        return carbonDictionaryColumnMetaChunksList;
-      }
-
-    };
-
-    new MockUp<ThriftReader>() {
-      @Mock public void setReadOffset(long bytesToSkip) throws IOException {
-
-      }
-
-    };
-    new MockUp<ThriftReader>() {
-      @Mock public void open() throws IOException {
-      }
-
-    };
-    new MockUp<ThriftReader>() {
-      @Mock public boolean hasNext() throws IOException {
-        return true;
-      }
-
-    };
-    new MockUp<CarbonDictionaryColumnMetaChunk>() {
-      @Mock public long getStart_offset() {
-        return 0L;
-      }
-    };
-    new MockUp<ThriftReader>() {
-      @Mock public TBase read() throws IOException {
-        List<ByteBuffer> byteBufferList = new ArrayList<ByteBuffer>();
-        ByteBuffer buf = ByteBuffer.allocate(2);
-        byteBufferList.add(buf);
-        return new org.apache.carbondata.format.ColumnDictionaryChunk(byteBufferList);
-
-      }
-
-    };
-    int expectedResult = 20;
-    int actualResult = carbonDictionaryReaderImpl.read().size();
-    assertTrue(actualResult == expectedResult);
-
-  }
-
-}
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 a28986d..e69de29 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
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.reader.sortindex;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- *
- */
-public class CarbonDictionarySortIndexReaderImplTest {
-  private String storePath;
-
-  @Before public void setUp() throws Exception {
-      storePath = "target/carbonStore";
-  }
-
-  @After public void tearDown() throws Exception {
-
-    deleteStorePath();
-  }
-
-  /**
-   * Test to read the data from dictionary sort index file
-   *
-   * @throws Exception
-   */
-  @Test public void read() throws Exception {
-    deleteStorePath();
-    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("testSchema", "carbon",
-    		UUID.randomUUID().toString());
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        AbsoluteTableIdentifier.from(storePath+"/testSchema/carbon", carbonTableIdentifier);
-    ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    CarbonDictionaryWriter dictionaryWriter =
-        new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-    String metaFolderPath =
-        storePath + File.separator + carbonTableIdentifier.getDatabaseName() + File.separator
-            + carbonTableIdentifier.getTableName() + File.separator + "Metadata";
-    CarbonUtil.checkAndCreateFolder(metaFolderPath);
-    CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
-    List<int[]> expectedData = prepareExpectedData();
-    int[] data = expectedData.get(0);
-    for(int i=0;i<data.length;i++) {
-    	dictionaryWriter.write(String.valueOf(data[i]));
-    }
-    dictionaryWriter.close();
-    dictionaryWriter.commit();
-    List<Integer> sortIndex = Arrays.asList(ArrayUtils.toObject(expectedData.get(0)));
-    List<Integer> invertedSortIndex = Arrays.asList(ArrayUtils.toObject(expectedData.get(1)));
-    dictionarySortIndexWriter.writeSortIndex(sortIndex);
-    dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
-    dictionarySortIndexWriter.close();
-    CarbonDictionarySortIndexReader dictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(dictionaryColumnUniqueIdentifier);
-    List<Integer> actualSortIndex = dictionarySortIndexReader.readSortIndex();
-    List<Integer> actualInvertedSortIndex = dictionarySortIndexReader.readInvertedSortIndex();
-    for (int i = 0; i < actualSortIndex.size(); i++) {
-      Assert.assertEquals(sortIndex.get(i), actualSortIndex.get(i));
-      Assert.assertEquals(invertedSortIndex.get(i), actualInvertedSortIndex.get(i));
-    }
-
-  }
-
-  /**
-   * Method return the list of sortIndex and sortIndexInverted array
-   *
-   * @return
-   */
-  private List<int[]> prepareExpectedData() {
-    List<int[]> indexList = new ArrayList<>(2);
-    int[] sortIndex = { 0, 3, 2, 4, 1 };
-    int[] sortIndexInverted = { 0, 2, 4, 1, 2 };
-    indexList.add(0, sortIndex);
-    indexList.add(1, sortIndexInverted);
-    return indexList;
-  }
-
-  /**
-   * this method will delete the store path
-   */
-  private void deleteStorePath() {
-    CarbonFile carbonFile = FileFactory.getCarbonFile(this.storePath);
-    deleteRecursiveSilent(carbonFile);
-  }
-
-  /**
-   * this method will delete the folders recursively
-   */
-  private static void deleteRecursiveSilent(CarbonFile f) {
-    if (f.isDirectory()) {
-      if (f.listFiles() != null) {
-        for (CarbonFile c : f.listFiles()) {
-          deleteRecursiveSilent(c);
-        }
-      }
-    }
-    if (f.exists() && !f.delete()) {
-      return;
-    }
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
index 58cd2fc..7b5d9c1 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryTypeTest.java
@@ -22,11 +22,8 @@
 import org.apache.carbondata.core.cache.dictionary.ColumnDictionaryInfo;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.ForwardDictionary;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.keygenerator.mdkey.Bits;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.DataTypeUtil;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -89,37 +86,4 @@
     assertEquals(expectedValue, actualValue);
   }
 
-  @Test public void testGetDataBasedOnDataTypeFromSurrogatesWhenIsDictionaryFalse() {
-    ByteBuffer surrogateData = ByteBuffer.allocate(10);
-    surrogateData.put(3, (byte) 1);
-    new MockUp<ForwardDictionary>() {
-      @Mock public String getDictionaryValueForKey(int surrogateKey) {
-        return "2015-10-20 12:30:01";
-      }
-    };
-    Object expectedValue = primitiveQueryTypeForTimeStampForIsDictionaryFalse
-        .getDataBasedOnDataType(surrogateData);
-    Object actualValue = primitiveQueryTypeForTimeStampForIsDictionaryFalse
-        .getDataBasedOnDataType(surrogateData);
-    assertEquals(expectedValue, actualValue);
-  }
-
-  public Object getDataBasedOnDataTypeFromSurrogates(ByteBuffer surrogateData) {
-    int keySize = 2;
-    byte[] data = new byte[keySize];
-    surrogateData.get(data);
-    Bits bit = new Bits(new int[] { keySize * 8 });
-    int surrgateValue = (int) bit.getKeyArray(data, 0)[0];
-    Object actualData = null;
-    if (isDirectDictionary) {
-      DirectDictionaryGenerator directDictionaryGenerator =
-          DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
-      actualData = directDictionaryGenerator.getValueFromSurrogate(surrgateValue);
-    } else {
-      String dictionaryValueForKey = dictionary.getDictionaryValueForKey(surrgateValue);
-      actualData = DataTypeUtil.getDataBasedOnDataType(dictionaryValueForKey, DataTypes.TIMESTAMP);
-    }
-    return actualData;
-  }
-
 }
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessorTest.java
index d1fc75b..5387043 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessorTest.java
@@ -23,7 +23,6 @@
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.carbondata.core.cache.dictionary.AbstractDictionaryCacheTest;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -40,18 +39,11 @@
 import org.junit.Before;
 import org.junit.Test;
 
-public class FilterExpressionProcessorTest extends AbstractDictionaryCacheTest {
+public class FilterExpressionProcessorTest {
 
   private ColumnSchema columnSchema;
 
   @Before public void setUp() throws Exception {
-    init();
-    this.databaseName = props.getProperty("database", "testSchema");
-    this.tableName = props.getProperty("tableName", "carbon");
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    carbonTableIdentifier =
-        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     columnSchema = new ColumnSchema();
     columnSchema.setColumnName("IMEI");
     columnSchema.setColumnUniqueId(UUID.randomUUID().toString());
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 2c25efb..486c341 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -21,13 +21,11 @@
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.carbondata.core.cache.dictionary.AbstractDictionaryCacheTest;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -54,18 +52,11 @@
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
-public class FilterUtilTest extends AbstractDictionaryCacheTest {
+public class FilterUtilTest {
 
   private ColumnSchema columnSchema;
 
   @Before public void setUp() throws Exception {
-    init();
-    this.databaseName = props.getProperty("database", "testSchema");
-    this.tableName = props.getProperty("tableName", "carbon");
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    carbonTableIdentifier =
-        new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     columnSchema = new ColumnSchema();
     columnSchema.setColumnName("IMEI");
     columnSchema.setColumnUniqueId(UUID.randomUUID().toString());
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 b8f3984..e69de29 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
@@ -1,536 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-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.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
-import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
-import org.apache.carbondata.core.reader.CarbonDictionaryReaderImpl;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * This class will test the functionality writing and
- * reading a dictionary and its corresponding metadata file
- */
-public class CarbonDictionaryWriterImplTest {
-
-  private static final String PROPERTY_FILE_NAME = "carbonTest.properties";
-
-  private CarbonTableIdentifier carbonTableIdentifier;
-
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-
-  private String databaseName;
-
-  private String tableName;
-
-  private String tablePath;
-
-  private ColumnIdentifier columnIdentifier;
-
-  private DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier;
-
-  private Properties props;
-
-  /**
-   * dictionary file path
-   */
-  private String dictionaryFilePath;
-
-  /**
-   * dictionary metadata file path
-   */
-  private String dictionaryMetaFilePath;
-
-  private List<String> dataSet1;
-
-  private List<String> dataSet2;
-
-  private List<String> dataSet3;
-
-  @Before public void setUp() throws Exception {
-    init();
-    this.databaseName = props.getProperty("database", "testSchema");
-    this.tableName = props.getProperty("tableName", "carbon");
-    this.tablePath = props.getProperty("storePath", "carbonStore");
-    this.columnIdentifier = new ColumnIdentifier("Name", null, null);
-    carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, carbonTableIdentifier);
-    this.dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    deleteStorePath();
-    prepareDataSet();
-  }
-
-  @After public void tearDown() throws Exception {
-    carbonTableIdentifier = null;
-    deleteStorePath();
-  }
-
-  /**
-   * prepare the dataset required for running test cases
-   */
-  private void prepareDataSet() {
-    dataSet1 = Arrays.asList("a", "b");
-    dataSet2 = Arrays.asList("c", "d");
-    dataSet3 = Arrays.asList("e", "f");
-  }
-
-  /**
-   * test writers write functionality for a column specific
-   * to a table in a database
-   */
-  @Test public void testWriteForNormalColumn() throws IOException {
-    // second parameter is chunk count which is for the number of
-    // thrift objects written for a segment
-    processColumnValuesForOneChunk(1);
-  }
-
-  /**
-   * test writers write functionality for a column shared across tables
-   * in a database
-   */
-  @Test public void testWriteForSharedColumn() throws IOException {
-    // second parameter is chunk count which is for the number of
-    // thrift objects written for a segment
-    processColumnValuesForOneChunk(1);
-  }
-
-  /**
-   * test writing multiple dictionary chunks for a single segment
-   */
-  @Test public void testWriteMultipleChunksForOneSegment() throws IOException {
-    deleteStorePath();
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE, "1");
-    // prepare dictionary writer object
-    CarbonDictionaryWriterImpl writer = prepareWriter();
-    writeDictionaryFile(writer, dataSet1);
-    // record file size from where data has to be read
-    long end_offset = CarbonUtil.getFileSize(this.dictionaryFilePath);
-    // read metadata chunks from file
-    List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
-        readDictionaryMetadataFile();
-    assertTrue(1 == carbonDictionaryColumnMetaChunks.size());
-    // prepare retrieved chunk metadata
-    long start_offset = 0L;
-    CarbonDictionaryColumnMetaChunk expected =
-        new CarbonDictionaryColumnMetaChunk(1, dataSet1.size(), start_offset, end_offset,
-            dataSet1.size());
-    // validate chunk metadata - actual and expected
-    for (CarbonDictionaryColumnMetaChunk chunk : carbonDictionaryColumnMetaChunks) {
-      validateDictionaryMetadata(chunk, expected);
-    }
-    //assert for chunk count
-    List<byte[]> dictionaryValues = readDictionaryFile(0L, 0L);
-    // prepare expected dictionary chunk list
-    List<String> actual = convertByteArrayListToStringValueList(dictionaryValues);
-    assertTrue(dataSet1.size() == actual.size());
-    // validate the dictionary data
-    compareDictionaryData(actual, dataSet1);
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE,
-        CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT);
-  }
-
-  /**
-   * prepare the dictionary writer object
-   */
-  private CarbonDictionaryWriterImpl prepareWriter() throws IOException {
-    initDictionaryDirPaths();
-    return new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * this method will test the write method in case of any exception
-   */
-  @Test public void testWriteThrowException() throws IOException {
-    final String failureMessage = "write operation failed";
-    // mock write method of writer and throw exception
-    new MockUp<CarbonDictionaryWriterImpl>() {
-      @Mock public void write(String value) throws IOException {
-        throw new IOException(failureMessage);
-      }
-    };
-    // prepare the writer
-    CarbonDictionaryWriterImpl writer = prepareWriter();
-    try {
-      for (String value : dataSet1) {
-        // exception should be thrown when write method is called
-        writer.write(value);
-      }
-    } catch (IOException e) {
-      assertTrue(failureMessage.equals(e.getMessage()));
-    } finally {
-      writer.close();
-    }
-  }
-
-  /**
-   * This method will test the truncate functionality
-   */
-  @Test public void testTruncateOperation() throws IOException {
-    // delete store path
-    deleteStorePath();
-    // prepare first dictionary chunk
-    // prepare dictionary writer object
-    CarbonDictionaryWriterImpl writer = prepareWriter();
-    writeDictionaryFile(writer, dataSet1);
-    long endOffsetAfterFirstDictionaryChunk = CarbonUtil.getFileSize(dictionaryFilePath);
-    // maintain the offset till end offset of first chunk
-    writer = prepareWriter();
-    writeDictionaryFile(writer, dataSet2);
-    // prepare first column meta chunk object
-    ColumnDictionaryChunkMeta firstDictionaryChunkMeta =
-        new ColumnDictionaryChunkMeta(1, 2, 0, endOffsetAfterFirstDictionaryChunk, 1);
-    // overwrite the dictionary meta chunk file to test the truncate operation
-    overwriteDictionaryMetaFile(firstDictionaryChunkMeta, dictionaryMetaFilePath);
-    writer = prepareWriter();
-    // in the next step truncate operation will be tested while writing dictionary file
-    writeDictionaryFile(writer, dataSet3);
-    // read dictionary file
-    List<byte[]> dictionaryValues = readDictionaryFile(0L, 0L);
-    List<String> actual = convertByteArrayListToStringValueList(dictionaryValues);
-    List<String> expected = new ArrayList<>(4);
-    expected.addAll(dataSet1);
-    expected.addAll(dataSet3);
-    // validate the data retrieved and it should match dataset1
-    compareDictionaryData(actual, expected);
-  }
-
-  /**
-   * This method will overwrite a given file with data provided
-   */
-  private void overwriteDictionaryMetaFile(ColumnDictionaryChunkMeta firstDictionaryChunkMeta,
-      String dictionaryFile) throws IOException {
-    ThriftWriter thriftMetaChunkWriter = new ThriftWriter(dictionaryFile, false);
-    try {
-      thriftMetaChunkWriter.open();
-      thriftMetaChunkWriter.write(firstDictionaryChunkMeta);
-    } catch (IOException e) {
-
-    } finally {
-      thriftMetaChunkWriter.close();
-    }
-
-  }
-
-  /**
-   * this method will test the reading of dictionary file from a given offset
-   */
-  @Test public void testReadingOfDictionaryChunkFromAnOffset() throws Exception {
-    // delete store path
-    deleteStorePath();
-    // prepare the writer to write dataset1
-    CarbonDictionaryWriterImpl writer = prepareWriter();
-    // write dataset1 data
-    writeDictionaryFile(writer, dataSet1);
-    // prepare the writer to write dataset2
-    writer = prepareWriter();
-    // write dataset2
-    writeDictionaryFile(writer, dataSet2);
-    // record the offset from where data has to be read
-    long dictionaryFileOffsetToRead = CarbonUtil.getFileSize(this.dictionaryFilePath);
-    // prepare writer to write dataset3
-    writer = prepareWriter();
-    // write dataset 3
-    writeDictionaryFile(writer, dataSet3);
-    // read dictionary chunk from dictionary file
-    List<byte[]> dictionaryData = readDictionaryFile(dictionaryFileOffsetToRead, 0L);
-    // prepare the retrieved data
-    List<String> actual = convertByteArrayListToStringValueList(dictionaryData);
-    // compare dictionary data set
-    compareDictionaryData(actual, dataSet3);
-    // read chunk metadata file
-    List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
-        readDictionaryMetadataFile();
-    // assert for metadata chunk size
-    assertTrue(3 == carbonDictionaryColumnMetaChunks.size());
-  }
-
-  /**
-   * this method will test the reading of dictionary file between start and end offset
-   */
-  @Test public void testReadingOfDictionaryChunkBetweenStartAndEndOffset() throws Exception {
-    // delete store path
-    deleteStorePath();
-    // prepare the writer to write dataset1
-    CarbonDictionaryWriterImpl writer = prepareWriter();
-    // write dataset1 data
-    writeDictionaryFile(writer, dataSet1);
-    // record dictionary file start offset
-    long dictionaryStartOffset = CarbonUtil.getFileSize(this.dictionaryFilePath);
-    // prepare the writer to write dataset2
-    writer = prepareWriter();
-    // write dataset2
-    writeDictionaryFile(writer, dataSet2);
-    // record the end offset for dictionary file
-    long dictionaryFileEndOffset = CarbonUtil.getFileSize(this.dictionaryFilePath);
-    // prepare writer to write dataset3
-    writer = prepareWriter();
-    // write dataset 3
-    writeDictionaryFile(writer, dataSet3);
-    // read dictionary chunk from dictionary file
-    List<byte[]> dictionaryData =
-        readDictionaryFile(dictionaryStartOffset, dictionaryFileEndOffset);
-    // prepare the retrieved data
-    List<String> actual = convertByteArrayListToStringValueList(dictionaryData);
-    // compare dictionary data set
-    compareDictionaryData(actual, dataSet2);
-    // read chunk metadata file
-    List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
-        readDictionaryMetadataFile();
-    // assert for metadata chunk size
-    assertTrue(3 == carbonDictionaryColumnMetaChunks.size());
-    CarbonDictionaryColumnMetaChunk expected =
-        new CarbonDictionaryColumnMetaChunk(3, 4, dictionaryStartOffset, dictionaryFileEndOffset,
-            1);
-    validateDictionaryMetadata(carbonDictionaryColumnMetaChunks.get(1), expected);
-  }
-
-  /**
-   * This method will convert list of byte array to list of string
-   */
-  private List<String> convertByteArrayListToStringValueList(List<byte[]> dictionaryByteArrayList) {
-    List<String> valueList = new ArrayList<>(dictionaryByteArrayList.size());
-    for (byte[] value : dictionaryByteArrayList) {
-      valueList.add(new String(value, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-    }
-    return valueList;
-  }
-
-  /**
-   * this method will write the data into a file
-   */
-  private void writeDictionaryFile(CarbonDictionaryWriterImpl writer, List<String> list)
-      throws IOException {
-    try {
-      for (String value : list) {
-        writer.write(value);
-      }
-    } finally {
-      writer.close();
-      writer.commit();
-    }
-  }
-
-  /**
-   * this method will test the functionality of writing and reading one dictionary chunk
-   */
-  private void processColumnValuesForOneChunk(int chunkCountForSegment) throws IOException {
-    // delete store path
-    deleteStorePath();
-    // prepare writer
-    CarbonDictionaryWriterImpl writer = prepareWriter();
-    // write the data into file
-    // test write api for passing list of byte array
-    writer.write(convertStringListToByteArray(dataSet1));
-    // close the writer
-    writer.close();
-    //write metadata
-    writer.commit();
-    // record end offset of file
-    long end_offset = CarbonUtil.getFileSize(this.dictionaryFilePath);
-    // read dictionary chunk from dictionary file
-    List<byte[]> dictionaryData = readDictionaryFile(0L, 0L);
-    // prepare the retrieved data
-    List<String> actual = convertByteArrayListToStringValueList(dictionaryData);
-    // compare the expected and actual data
-    compareDictionaryData(actual, dataSet1);
-    // read dictionary metadata chunks
-    List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
-        readDictionaryMetadataFile();
-    // assert
-    assertTrue(1 == carbonDictionaryColumnMetaChunks.size());
-    long start_offset = 0L;
-    // validate actual chunk metadata with expected
-    CarbonDictionaryColumnMetaChunk expected =
-        new CarbonDictionaryColumnMetaChunk(1, 2, start_offset, end_offset, 1);
-    for (CarbonDictionaryColumnMetaChunk chunk : carbonDictionaryColumnMetaChunks) {
-      validateDictionaryMetadata(chunk, expected);
-    }
-  }
-
-  /**
-   * this method will convert list of string to list of byte array
-   */
-  private List<byte[]> convertStringListToByteArray(List<String> valueList) {
-    List<byte[]> byteArrayList = new ArrayList<>(valueList.size());
-    for (String value : valueList) {
-      byteArrayList.add(value.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-    }
-    return byteArrayList;
-  }
-
-  /**
-   * this method will validate the dictionary chunk metadata
-   */
-  private void validateDictionaryMetadata(CarbonDictionaryColumnMetaChunk actual,
-      CarbonDictionaryColumnMetaChunk expected) {
-    assertTrue(expected.getMin_surrogate_key() == actual.getMin_surrogate_key());
-    assertTrue(expected.getMax_surrogate_key() == actual.getMax_surrogate_key());
-    assertTrue(expected.getStart_offset() == actual.getStart_offset());
-    assertTrue(expected.getEnd_offset() == actual.getEnd_offset());
-    assertTrue(expected.getChunk_count() == actual.getChunk_count());
-  }
-
-  /**
-   * this method will validate the dictionary data
-   */
-  private void compareDictionaryData(List<String> actual, List<String> expected) {
-    assertTrue(expected.size() == actual.size());
-    for (int i = 0; i < actual.size(); i++) {
-      assertTrue(actual.get(i).equals(expected.get(i)));
-    }
-  }
-
-  /**
-   * This method will read dictionary metadata file and return the dictionary meta chunks
-   *
-   * @return list of dictionary metadata chunks
-   * @throws IOException read and close method throws IO excpetion
-   */
-  private List<CarbonDictionaryColumnMetaChunk> readDictionaryMetadataFile() throws IOException {
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.dictionaryColumnUniqueIdentifier);
-    List<CarbonDictionaryColumnMetaChunk> dictionaryMetaChunkList = null;
-    // read metadata file
-    try {
-      dictionaryMetaChunkList = columnMetadataReaderImpl.read();
-    } finally {
-      // close the metadata reader
-      columnMetadataReaderImpl.close();
-    }
-    return dictionaryMetaChunkList;
-  }
-
-  /**
-   * This method will be used to read the dictionary file from a given offset
-   */
-  private List<byte[]> readDictionaryFile(long dictionaryStartOffset, long dictionaryEndOffset)
-      throws IOException {
-    CarbonDictionaryReaderImpl dictionaryReader =
-        new CarbonDictionaryReaderImpl(this.dictionaryColumnUniqueIdentifier);
-    List<byte[]> dictionaryValues = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    try {
-      if (0 == dictionaryEndOffset) {
-        dictionaryValues = dictionaryReader.read(dictionaryStartOffset);
-      } else {
-        Iterator<byte[]> itr = dictionaryReader.read(dictionaryStartOffset, dictionaryEndOffset);
-        while (itr.hasNext()) {
-          dictionaryValues.add(itr.next());
-        }
-      }
-    } finally {
-      dictionaryReader.close();
-    }
-    return dictionaryValues;
-  }
-
-  /**
-   * this method will delete the store path
-   */
-  private void deleteStorePath() {
-    CarbonFile carbonFile = FileFactory.getCarbonFile(this.tablePath);
-    deleteRecursiveSilent(carbonFile);
-  }
-
-  /**
-   * this method will delete the folders recursively
-   */
-  private static void deleteRecursiveSilent(CarbonFile f) {
-    if (f.isDirectory()) {
-      if (f.listFiles() != null) {
-        for (CarbonFile c : f.listFiles()) {
-          deleteRecursiveSilent(c);
-        }
-      }
-    }
-    if (f.exists() && !f.delete()) {
-      return;
-    }
-  }
-
-  /**
-   * this method will read the property file for required details
-   * like dbName, tableName, etc
-   */
-  private void init() {
-    InputStream in = null;
-    props = new Properties();
-    try {
-      URL url = getClass().getClassLoader().getResource(PROPERTY_FILE_NAME);
-      if (null != url) {
-        URI uri = url.toURI();
-        File file = new File(uri);
-        in = new FileInputStream(file);
-        props.load(in);
-      }
-    } catch (IOException e) {
-      e.printStackTrace();
-    } catch (URISyntaxException e) {
-      e.printStackTrace();
-    } finally {
-      CarbonUtil.closeStreams(in);
-    }
-  }
-
-  /**
-   * this method will form the dictionary directory paths
-   */
-  private void initDictionaryDirPaths() throws IOException {
-    String dictionaryLocation = CarbonTablePath.getMetadataPath(tablePath);
-    if(!FileFactory.isFileExist(dictionaryLocation)) {
-      FileFactory.mkdirs(dictionaryLocation);
-    }
-    this.dictionaryFilePath = CarbonTablePath.getDictionaryFilePath(tablePath, columnIdentifier.getColumnId());
-    this.dictionaryMetaFilePath = CarbonTablePath.getDictionaryMetaFilePath(tablePath, columnIdentifier.getColumnId());
-  }
-}
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
deleted file mode 100644
index d9081ac..0000000
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
-import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
-
-import org.apache.commons.lang.ArrayUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * class contains the unit test cases of the dictionary sort index & sort index inverted writing
- */
-public class CarbonDictionarySortIndexWriterImplTest {
-
-  private String storePath;
-  private CarbonTableIdentifier carbonTableIdentifier = null;
-  private AbsoluteTableIdentifier absoluteTableIdentifier = null;
-  private ColumnIdentifier columnIdentifier = null;
-  private CarbonDictionaryWriter dictionaryWriter = null;
-  private CarbonDictionarySortIndexWriter dictionarySortIndexWriter = null;
-  private CarbonDictionarySortIndexReader carbonDictionarySortIndexReader = null;
-
-  @Before public void setUp() throws Exception {
-    storePath = "target/carbonStore";
-    carbonTableIdentifier =
-        new CarbonTableIdentifier("testSchema", "carbon", UUID.randomUUID().toString());
-    String tablePath =
-        storePath + "/" + carbonTableIdentifier.getDatabaseName() + "/" + carbonTableIdentifier
-            .getTableName();
-    absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, carbonTableIdentifier);
-    columnIdentifier = new ColumnIdentifier("Name", null, null);
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
-            columnIdentifier.getDataType());
-    dictionaryWriter =
-        new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-    dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
-    carbonDictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(dictionaryColumnUniqueIdentifier);
-  }
-
-  /**
-   * s
-   * Method to test the write of sortIndex file.
-   *
-   * @throws Exception
-   */
-  @Test public void write() throws Exception {
-
-    String metaFolderPath =
-        storePath + File.separator + carbonTableIdentifier.getDatabaseName() + File.separator
-            + carbonTableIdentifier.getTableName() + File.separator + "Metadata";
-    CarbonUtil.checkAndCreateFolder(metaFolderPath);
-
-    List<int[]> indexList = prepareExpectedData();
-    int[] data = indexList.get(0);
-    for (int i = 0; i < data.length; i++) {
-      dictionaryWriter.write(String.valueOf(data[i]));
-    }
-    dictionaryWriter.close();
-    dictionaryWriter.commit();
-
-    List<Integer> sortIndex = Arrays.asList(ArrayUtils.toObject(indexList.get(0)));
-    List<Integer> invertedSortIndex = Arrays.asList(ArrayUtils.toObject(indexList.get(1)));
-    dictionarySortIndexWriter.writeSortIndex(sortIndex);
-    dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
-    dictionarySortIndexWriter.close();
-
-    List<Integer> actualSortIndex = carbonDictionarySortIndexReader.readSortIndex();
-    List<Integer> actualInvertedSortIndex = carbonDictionarySortIndexReader.readInvertedSortIndex();
-    for (int i = 0; i < actualSortIndex.size(); i++) {
-      assertEquals(sortIndex.get(i), actualSortIndex.get(i));
-      assertEquals(invertedSortIndex.get(i), actualInvertedSortIndex.get(i));
-    }
-
-  }
-
-  /**
-   * @throws Exception
-   */
-  @Test public void writingEmptyValue() throws Exception {
-
-    List<Integer> sortIndex = new ArrayList<>();
-    List<Integer> invertedSortIndex = new ArrayList<>();
-    dictionarySortIndexWriter.writeSortIndex(sortIndex);
-    dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
-    dictionarySortIndexWriter.close();
-    List<Integer> actualSortIndex = carbonDictionarySortIndexReader.readSortIndex();
-    List<Integer> actualInvertedSortIndex = carbonDictionarySortIndexReader.readInvertedSortIndex();
-    for (int i = 0; i < actualSortIndex.size(); i++) {
-      assertEquals(sortIndex.get(i), actualSortIndex.get(i));
-      assertEquals(invertedSortIndex.get(i), actualInvertedSortIndex.get(i));
-    }
-
-  }
-
-  private List<int[]> prepareExpectedData() {
-    List<int[]> indexList = new ArrayList<>(2);
-    int[] sortIndex = { 0, 3, 2, 4, 1 };
-    int[] sortIndexInverted = { 0, 2, 4, 1, 2 };
-    indexList.add(0, sortIndex);
-    indexList.add(1, sortIndexInverted);
-    return indexList;
-  }
-
-}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
deleted file mode 100644
index 02f5b85..0000000
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortInfoPreparatorTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryChunksWrapper;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * The class tests the CarbonDictionarySortInfoPreparator class that prepares the column sort info ie sortIndex
- * and inverted sort index info
- */
-public class CarbonDictionarySortInfoPreparatorTest {
-
-  private static CarbonDictionarySortInfoPreparator carbonDictionarySortInfoPreparator = null;
-
-  @BeforeClass public static void setUp() {
-    carbonDictionarySortInfoPreparator = new CarbonDictionarySortInfoPreparator();
-  }
-
-  /**
-   * Tests the getDictionarySortInfo method
-   */
-  @Test public void testGetDictionarySortInfo() {
-
-    List<String> newDistinctValues = new ArrayList<>();
-    newDistinctValues.add("abc");
-    newDistinctValues.add("xyz");
-    Dictionary dictionary = new MockUp<Dictionary>() {
-      @Mock public DictionaryChunksWrapper getDictionaryChunks() {
-        List<byte[]> data = new ArrayList<>();
-        data.add(new byte[] { 1, 2 });
-        List<List<byte[]>> dictionaryChunks = new ArrayList<>();
-        dictionaryChunks.add(data);
-        return new DictionaryChunksWrapper(dictionaryChunks);
-      }
-    }.getMockInstance();
-
-    new MockUp<DictionaryChunksWrapper>() {
-      @Mock public int getSize() {
-        return 1;
-      }
-    };
-
-    CarbonDictionarySortInfo carbonDictionarySortInfo = carbonDictionarySortInfoPreparator
-        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.STRING);
-    int expectedGetSortIndexValue = 1;
-    int expectedGetSortInvertedIndexLength = 3;
-    int actualGetSortIndexValue = carbonDictionarySortInfo.getSortIndex().get(0);
-    int actualGetSortInvertedIndexLength = carbonDictionarySortInfo.getSortIndexInverted().size();
-    assertEquals(actualGetSortIndexValue, expectedGetSortIndexValue);
-    assertEquals(actualGetSortInvertedIndexLength, expectedGetSortInvertedIndexLength);
-  }
-
-  /**
-   * Tests getDictionarySortInfo when dictionary is null
-   */
-  @Test public void testGetDictionarySortInfoDictionaryNullCase() {
-
-    List<String> newDistinctValues = new ArrayList<>();
-    newDistinctValues.add("abc");
-    newDistinctValues.add("xyz");
-    Dictionary dictionary = null;
-    CarbonDictionarySortInfo carbonDictionarySortInfo = carbonDictionarySortInfoPreparator
-        .getDictionarySortInfo(newDistinctValues, dictionary, DataTypes.createDefaultArrayType());
-    int expectedGetSortIndexValue = 1;
-    int expectedGetSortInvertedIndexLength = 2;
-    int actualGetSortIndexValue = carbonDictionarySortInfo.getSortIndex().get(0);
-    int actualGetSortInvertedIndexLength = carbonDictionarySortInfo.getSortIndexInverted().size();
-    assertEquals(actualGetSortIndexValue, expectedGetSortIndexValue);
-    assertEquals(actualGetSortInvertedIndexLength, expectedGetSortInvertedIndexLength);
-  }
-
-}
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java b/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
deleted file mode 100644
index cd7c794..0000000
--- a/core/src/test/java/org/apache/carbondata/core/writer/sortindex/CarbonDictionarySortModelTest.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.writer.sortindex;
-
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-/**
- * This class tests the Dictionary sort model class that holds the member byte value and corresponding key value.
- */
-public class CarbonDictionarySortModelTest {
-
-  private CarbonDictionarySortModel carbonDictionarySortModel = null;
-
-  @Test public void testCompareToForDataTypeDoubleCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "7234");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "5678");
-    int expectedResult = 1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeDoubleExceptionCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "double");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "@NU#LL$!");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeDoubleExceptionCaseForOtherObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "1234");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "@NU#LL$!");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeBooleanCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "memberValue");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "value");
-    int expectedResult = -9;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeDecimalCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "72.34");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "56.78");
-    int expectedResult = 1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeDecimalExceptionCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "decimal");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "@NU#LL$!");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeDecimalExceptionCaseForOtherObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "15.24");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "@NU#LL$!");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeTimestampCase() {
-    carbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09-22 12:00:00");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "2015-09-22 12:08:49");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeTimestampExceptionCase() {
-    carbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09 12:00:00");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "@NU#LL$!");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeTimestampExceptionCaseForOtherObject() {
-    carbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09-22 12:00:00");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "2014-09-22 12");
-    int expectedResult = -1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testHashCode() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "15.24");
-    int actualResult = carbonDictionarySortModel.hashCode();
-    int expectedResult = 46877260;
-    assertTrue(actualResult == expectedResult);
-  }
-
-  @Test public void testHashCodeNullCaseForMemberValue() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), null);
-    int actualResult = carbonDictionarySortModel.hashCode();
-    int expectedResult = 0;
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testEquals() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "15.24");
-    CarbonDictionarySortModel testCarbonDictionarySortModel = carbonDictionarySortModel;
-    boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
-    assertTrue(result);
-  }
-
-  @Test public void testEqualsMemberValueNullCase() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), null);
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "false");
-    boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
-    assertFalse(result);
-  }
-
-  @Test public void testEqualsWhenMemberValueDiffers() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.BOOLEAN, "false");
-    boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
-    assertFalse(result);
-  }
-
-  @Test public void testEqualsWhenMemberValueIsSame() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
-    boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
-    assertTrue(result);
-  }
-
-  @Test public void testEqualsForDifferentObjects() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.45");
-    Object testCarbonDictionarySortModel = new Object();
-    boolean result = carbonDictionarySortModel.equals(testCarbonDictionarySortModel);
-    assertFalse(result);
-  }
-
-  @Test public void testCompareToForDataTypeDoubleExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.DOUBLE, "double");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.DOUBLE, "1234");
-    int expectedResult = 1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeDecimalExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.createDefaultDecimalType(), "12.il");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.createDefaultDecimalType(), "12.89");
-    int expectedResult = 1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-  @Test public void testCompareToForDataTypeTimestampExceptionCaseForDifferentObject() {
-    carbonDictionarySortModel = new CarbonDictionarySortModel(1, DataTypes.TIMESTAMP, "2014-09");
-    CarbonDictionarySortModel testCarbonDictionarySortModel =
-        new CarbonDictionarySortModel(2, DataTypes.TIMESTAMP, "2014-09-22 12:00:00");
-    int expectedResult = 1;
-    int actualResult = carbonDictionarySortModel.compareTo(testCarbonDictionarySortModel);
-    assertEquals(actualResult, expectedResult);
-  }
-
-}
diff --git a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
index 882ee45..aed282e 100644
--- a/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
+++ b/datamap/bloom/src/main/java/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMap.java
@@ -26,7 +26,6 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -40,7 +39,6 @@
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 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;
@@ -114,32 +112,23 @@
       this.name2Col.put(col.getColName(), col);
     }
 
-    try {
-      this.name2Converters = new HashMap<>(indexedColumn.size());
-      AbsoluteTableIdentifier absoluteTableIdentifier = AbsoluteTableIdentifier
-          .from(carbonTable.getTablePath(), carbonTable.getCarbonTableIdentifier());
-      String nullFormat = "\\N";
-      Map<Object, Integer>[] localCaches = new Map[indexedColumn.size()];
+    this.name2Converters = new HashMap<>(indexedColumn.size());
+    String nullFormat = "\\N";
 
-      for (int i = 0; i < indexedColumn.size(); i++) {
-        localCaches[i] = new ConcurrentHashMap<>();
-        DataField dataField = new DataField(indexedColumn.get(i));
-        String dateFormat = CarbonProperties.getInstance().getProperty(
-            CarbonCommonConstants.CARBON_DATE_FORMAT,
-            CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
-        dataField.setDateFormat(dateFormat);
-        String tsFormat = CarbonProperties.getInstance().getProperty(
-            CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-        dataField.setTimestampFormat(tsFormat);
-        FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
-            .createFieldEncoder(dataField, absoluteTableIdentifier, i, nullFormat, null, false,
-                localCaches[i], false, carbonTable.getTablePath(), false);
-        this.name2Converters.put(indexedColumn.get(i).getColName(), fieldConverter);
-      }
-    } catch (IOException e) {
-      LOGGER.error("Exception occurs while init index columns", e);
-      throw new RuntimeException(e);
+    for (int i = 0; i < indexedColumn.size(); i++) {
+      DataField dataField = new DataField(indexedColumn.get(i));
+      String dateFormat = CarbonProperties.getInstance().getProperty(
+          CarbonCommonConstants.CARBON_DATE_FORMAT,
+          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+      dataField.setDateFormat(dateFormat);
+      String tsFormat = CarbonProperties.getInstance().getProperty(
+          CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+      dataField.setTimestampFormat(tsFormat);
+      FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
+          .createFieldEncoder(dataField, i, nullFormat, false,
+              false, carbonTable.getTablePath());
+      this.name2Converters.put(indexedColumn.get(i).getColName(), fieldConverter);
     }
     this.badRecordLogHolder = new BadRecordLogHolder();
     this.badRecordLogHolder.setLogged(false);
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala
index f3e8091..3ddb0fc 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala
@@ -295,7 +295,7 @@
   }
 
   def validateDMProperty(tableProperty: mutable.Map[String, String]): Unit = {
-    val tableProperties = Array("dictionary_include", "dictionary_exclude", "sort_columns",
+    val tableProperties = Array("sort_columns",
       "local_dictionary_include", "local_dictionary_exclude", "long_string_columns",
       "no_inverted_index", "inverted_index", "column_meta_cache", "range_column")
     val unsupportedProps = tableProperty
diff --git a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala
index 19170c5..7ca5b12 100644
--- a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala
+++ b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/TestAllOperationsOnMV.scala
@@ -20,7 +20,7 @@
 import scala.collection.JavaConverters._
 import java.util
 
-import org.apache.spark.sql.{CarbonEnv, Row}
+import org.apache.spark.sql.{AnalysisException, CarbonEnv, Row}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterEach
@@ -405,17 +405,6 @@
     checkExistence(sql("select max(to_date(dob)) , min(to_date(dob)) from maintable where to_date(dob)='1975-06-11' or to_date(dob)='1975-06-23'"), true, "1975-06-11 1975-06-11")
   }
 
-  test("test global dictionary inherited from parent table") {
-    sql("drop table IF EXISTS maintable")
-    sql("create table maintable(name string, c_code int, price int) stored by 'carbondata' tblproperties('dictionary_include'='name')")
-    sql("insert into table maintable select 'abc',21,2000")
-    sql("drop datamap if exists dm ")
-    sql("create datamap dm using 'mv' as select name, sum(price) from maintable group by name")
-    checkExistence(sql("describe formatted dm_table"), true, "Global Dictionary maintable_name")
-    checkAnswer(sql("select name, sum(price) from maintable group by name"), Seq(Row("abc", 2000)))
-    sql("drop table IF EXISTS maintable")
-  }
-
   test("test preagg and mv") {
     sql("drop table IF EXISTS maintable")
     sql("create table maintable(name string, c_code int, price int) stored by 'carbondata'")
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 16fdf68..1d92a9a 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -89,8 +89,6 @@
 | carbon.load.skewedDataOptimization.enabled | false | During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks. This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance. In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading. When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.**NOTE:** This configuration is useful if the size of your input data files varies widely, say 1MB to 1GB. For this configuration to work effectively,knowing the data pattern and size is important and necessary. |
 | enable.data.loading.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues. This configuration when made ***true*** would log additional data loading statistics information to more accurately locate the issues being debugged. **NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time. It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately. Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced. Therefore it is recommended to enable this configuration only for the duration of debugging. |
 | carbon.dictionary.chunk.size | 10000 | CarbonData generates dictionary keys and writes them to separate dictionary file during data loading. To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time. **NOTE:** Writing to file also serves as a commit point to the dictionary generated. Increasing more values in memory causes more data loss during system or application failure. It is advised to alter this configuration judiciously. |
-| dictionary.worker.threads | 1 | CarbonData supports Optimized data loading by relying on a dictionary server. Dictionary server helps to maintain dictionary values independent of the data loading and there by avoids reading the same input data multiples times. This configuration determines the number of concurrent dictionary generation or request that needs to be served by the dictionary server. **NOTE:** This configuration takes effect when ***carbon.options.single.pass*** is configured as true. Please refer to *carbon.options.single.pass*to understand how dictionary server optimizes data loading. |
-| carbon.dictionary.server.port | 2030 | Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary. Single pass loading can be enabled using the option ***carbon.options.single.pass***. When this option is specified, a dictionary server will be internally started to handle the dictionary generation and query requests. This configuration specifies the port on which the server need to listen for incoming requests. Port value ranges between 0-65535 |
 | carbon.load.directWriteToStorePath.enabled | false | During data load, all the carbondata files are written to local disk and finally copied to the target store location in HDFS/S3. Enabling this parameter will make carbondata files to be written directly onto target HDFS/S3 location bypassing the local disk.**NOTE:** Writing directly to HDFS/S3 saves local disk IO(once for writing the files and again for copying to HDFS/S3) there by improving the performance. But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS/S3 location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
 | carbon.options.serialization.null.format | \N | Based on the business scenarios, some columns might need to be loaded with null values. As null value cannot be written in csv files, some special characters might be adopted to specify null values. This configuration can be used to specify the null values format in the data being loaded. |
 | carbon.column.compressor | snappy | CarbonData will compress the column values using the compressor specified by this configuration. Currently CarbonData supports 'snappy', 'zstd' and 'gzip' compressors. |
@@ -209,7 +207,6 @@
 | carbon.options.bad.records.action         | This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. |
 | carbon.options.is.empty.data.bad.record   | If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. |
 | carbon.options.batch.sort.size.inmb       | Size of batch data to keep in memory, as a thumb rule it supposed to be less than 45% of sort.inmemory.size.inmb otherwise it may spill intermediate data to disk. |
-| carbon.options.single.pass                | Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary. This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE. **NOTE:** Enabling this starts a new dictionary server to handle dictionary generation requests during data loading. Without this option, the input csv files will have to read twice. Once while dictionary generation and persisting to the dictionary files.second when the data loading need to convert the input data into carbondata format. Enabling this optimizes the optimizes to read the input data only once there by reducing IO and hence over all data loading time. If concurrent data loading needs to be supported, consider tuning ***dictionary.worker.threads***. Port on which the dictionary server need to listen on can be configured using the configuration ***carbon.dictionary.server.port***. |
 | carbon.options.bad.record.path            | Specifies the HDFS path where bad records needs to be stored. |
 | carbon.custom.block.distribution          | Specifies whether to use the Spark or Carbon block distribution feature.**NOTE: **Refer to [Query Configuration](#query-configuration)#carbon.custom.block.distribution for more details on CarbonData scheduler. |
 | enable.unsafe.sort                        | Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance. |
diff --git a/docs/datamap/mv-datamap-guide.md b/docs/datamap/mv-datamap-guide.md
index b75243f..5298a27 100644
--- a/docs/datamap/mv-datamap-guide.md
+++ b/docs/datamap/mv-datamap-guide.md
@@ -85,11 +85,9 @@
     6. LOCAL_DICTIONARY_ENABLE
     7. LOCAL_DICTIONARY_THRESHOLD
     8. LOCAL_DICTIONARY_EXCLUDE
-    9. DICTIONARY_INCLUDE
-   10. DICTIONARY_EXCLUDE
-   11. INVERTED_INDEX
-   12. NO_INVERTED_INDEX
-   13. COLUMN_COMPRESSOR
+    9. INVERTED_INDEX
+   10. NO_INVERTED_INDEX
+   11. COLUMN_COMPRESSOR
 
  * Creating MV datamap with select query containing only project of all columns of maintable is unsupported 
       
@@ -100,7 +98,7 @@
    1. ```select a,b,c from x```
    2. ```select * from x```
  * TableProperties can be provided in DMProperties excluding LOCAL_DICTIONARY_INCLUDE,
-   LOCAL_DICTIONARY_EXCLUDE, DICTIONARY_INCLUDE, DICTIONARY_EXCLUDE, INVERTED_INDEX,
+   LOCAL_DICTIONARY_EXCLUDE, INVERTED_INDEX,
    NO_INVERTED_INDEX, SORT_COLUMNS, LONG_STRING_COLUMNS, RANGE_COLUMN & COLUMN_META_CACHE
  * TableProperty given in DMProperties will be considered for mv creation, eventhough if same
    property is inherited from parent table, which allows user to provide different tableproperties
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index a7d3b4d..fcae480 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -83,7 +83,6 @@
 
 | Property                                                     | Description                                                  |
 | ------------------------------------------------------------ | ------------------------------------------------------------ |
-| [DICTIONARY_INCLUDE](#dictionary-encoding-configuration)     | Columns for which dictionary needs to be generated           |
 | [NO_INVERTED_INDEX](#inverted-index-configuration)           | Columns to exclude from inverted index generation            |
 | [INVERTED_INDEX](#inverted-index-configuration)              | Columns to include for inverted index generation             |
 | [SORT_COLUMNS](#sort-columns-configuration)                  | Columns to include in sort and its order of sort             |
@@ -112,19 +111,6 @@
 
  Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.
 
-   - ##### Dictionary Encoding Configuration
-
-     Dictionary encoding is turned off for all columns by default from 1.3 onwards, you can use this command for including or excluding columns to do dictionary encoding.
-     Suggested use cases : do dictionary encoding for low cardinality columns, it might help to improve data compression ratio and performance.
-
-     ```
-     TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
-     ```
-
-     **NOTE**: 
-      * Dictionary Include/Exclude for complex child columns is not supported. Dictionary Include doesn't support binary data type.  
-      * Dictionary is global. Except global dictionary, there are local dictionary and non-dictionary in CarbonData.
-      
    - ##### Local Dictionary Configuration
 
    Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to [File structure of Carbondata](./file-structure-of-carbondata.md) for understanding about the file structure of carbondata and meaning of terms like blocklet.
@@ -171,7 +157,7 @@
 | ---------- | ------------- | ----------- |
 | LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by '***carbon.local.dictionary.enable***'.Local dictionary will be generated for all string/varchar/char columns unless LOCAL_DICTIONARY_INCLUDE, LOCAL_DICTIONARY_EXCLUDE is configured. |
 | LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000). **NOTE:** When LOCAL_DICTIONARY_THRESHOLD is defined for Complex columns, the count of distinct records of all child columns are summed up. |
-| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation. This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others. This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
+| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated. This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others. This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
 | LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated. This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others. This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
 
    **Fallback behavior:** 
@@ -203,9 +189,6 @@
    **NOTE:** 
 
    * We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads
-   * On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.
-   * When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.
-   * If want to use non-dictionary, users can set LOCAL_DICTIONARY_ENABLE as false and don't set DICTIONARY_INCLUDE.
       
    - ##### Inverted Index Configuration
 
@@ -463,7 +446,7 @@
      If you are using Carbon-SDK, you can specify the datatype of long string column as `varchar`.
      You can refer to SDKwriterTestCase for example.
 
-     **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
+     **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be sort_columns/complex columns.
 
    - ##### Compression for table
 
@@ -669,8 +652,7 @@
      This command is used to add a new column to the existing table.
      ```
      ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
-     TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
-     'DEFAULT.VALUE.COLUMN_NAME'='default_value')
+     TBLPROPERTIES('DEFAULT.VALUE.COLUMN_NAME'='default_value')
      ```
 
      Examples:
@@ -679,10 +661,6 @@
      ```
 
      ```
-     ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
-     ```
-
-     ```
      ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
      ```
       **NOTE:** Add Complex datatype columns is not supported.
@@ -866,7 +844,6 @@
                                 productNumber Int COMMENT 'unique serial number for product')
   COMMENT "This is table comment"
   STORED AS carbondata
-  TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
   ```
 
   You can also SET and UNSET table comment using ALTER command.
diff --git a/docs/dml-of-carbondata.md b/docs/dml-of-carbondata.md
index 190551a..e148bd0 100644
--- a/docs/dml-of-carbondata.md
+++ b/docs/dml-of-carbondata.md
@@ -59,12 +59,9 @@
 | [SKIP_EMPTY_LINE](#skip_empty_line)                     | Whether empty lines in input csv file should be skipped or loaded as null row |
 | [COMPLEX_DELIMITER_LEVEL_1](#complex_delimiter_level_1) | Starting delimiter for complex type data in input csv file   |
 | [COMPLEX_DELIMITER_LEVEL_2](#complex_delimiter_level_2) | Ending delimiter for complex type data in input csv file     |
-| [ALL_DICTIONARY_PATH](#all_dictionary_path)             | Path to read the dictionary data from all columns            |
-| [COLUMNDICT](#columndict)                               | Path to read the dictionary data from for particular column  |
 | [DATEFORMAT](#dateformattimestampformat)                | Format of date in the input csv file                         |
 | [TIMESTAMPFORMAT](#dateformattimestampformat)           | Format of timestamp in the input csv file                    |
 | [SORT_COLUMN_BOUNDS](#sort-column-bounds)               | How to partition the sort columns to make the evenly distributed |
-| [SINGLE_PASS](#single_pass)                             | When to enable single pass data loading                      |
 | [BAD_RECORDS_LOGGER_ENABLE](#bad-records-handling)      | Whether to enable bad records logging                        |
 | [BAD_RECORD_PATH](#bad-records-handling)                | Bad records logging path. Useful when bad record logging is enabled |
 | [BAD_RECORDS_ACTION](#bad-records-handling)             | Behavior of data loading when bad record is found            |
@@ -177,23 +174,6 @@
     OPTIONS('COMPLEX_DELIMITER_LEVEL_3'='\003')
     ```
 
-  - ##### ALL_DICTIONARY_PATH:
-
-    All dictionary files path.
-
-    ```
-    OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
-    ```
-
-  - ##### COLUMNDICT:
-
-    Dictionary file path for specified column.
-
-    ```
-    OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
-    ```
-    **NOTE:** ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.
-
   - ##### DATEFORMAT/TIMESTAMPFORMAT:
 
     Date and Timestamp format for specified column.
@@ -216,38 +196,9 @@
     **NOTE:**
     * SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.
     * Carbondata will use these bounds as ranges to process data concurrently during the final sort procedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.
-    * Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.
     * The option works better if your CPU usage during loading is low. If your current system CPU usage is high, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.
     * Users can find more information about this option in the description of PR1953.
 
-  - ##### SINGLE_PASS:
-
-    Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.
-
-  This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.
-
-   ```
-    OPTIONS('SINGLE_PASS'='TRUE')
-   ```
-
-   **NOTE:**
-   * If this option is set to TRUE then data loading will take less time.
-   * If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.
-
-   Example:
-
-   ```
-   LOAD DATA inpath '/opt/rawdata/data.csv' INTO table carbontable
-   options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
-   'HEADER'='false',
-   'FILEHEADER'='empno,empname,designation,doj,workgroupcategory,
-   workgroupcategoryname,deptno,deptname,projectcode,
-   projectjoindate,projectenddate,attendance,utilization,salary',
-   'MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='\\\001',
-   'COMPLEX_DELIMITER_LEVEL_2'='\\\002',
-   'ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary',
-   'SINGLE_PASS'='TRUE')
-   ```
 
   - ##### BAD RECORDS HANDLING:
 
diff --git a/docs/file-structure-of-carbondata.md b/docs/file-structure-of-carbondata.md
index 9313593..6ffc331 100644
--- a/docs/file-structure-of-carbondata.md
+++ b/docs/file-structure-of-carbondata.md
@@ -63,8 +63,6 @@
 
 When loading data in batches, each batch loading generates a new segment directory. The scheduling tries to control a task processing data loading task on each node. Each task will generate multiple carbondata files and one carbonindex file.
 
-During  global dictionary generation, if the two-pass scheme is used, before the data is loaded, the corresponding dict, dictmeta and sortindex files are generated for each dictionary-encoded column, and partial dictionary files can be provided by the pre-define dictionary method to reduce the need. A dictionary-encoded column is generated by scanning the full amount of data; a dictionary file of all dictionary code columns can also be provided by the all dictionary method to avoid scanning data. If the single-pass scheme is adopted, the global dictionary code is generated in real time during data loading, and after the data is loaded, the dictionary is solidified into a dictionary file.
-
 The following sections use the Java object generated by the thrift file describing the carbondata file format to explain the contents of each file one by one (you can also directly read the format defined in the [thrift file](https://github.com/apache/carbondata/tree/master/format/src/main/thrift))
 
 ### Schema file format
diff --git a/docs/usecases.md b/docs/usecases.md
index c270eef..343fccd 100644
--- a/docs/usecases.md
+++ b/docs/usecases.md
@@ -191,7 +191,6 @@
 - Use global sort mode so that the data to be fetched are grouped together
 - Create Materialized View for aggregation queries
 - Reduce the Spark shuffle partitions.(In our configuration on 14 node cluster, it was reduced to 35 from default of 200)
-- Enable global dictionary for columns which have less cardinalities. Aggregation can be done on encoded data, there by improving the performance
 - For columns whose cardinality is high,enable the local dictionary so that store size is less and can take dictionary benefit for scan
 
 ## Handling near realtime data ingestion scenario
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/ConcurrentQueryBenchmark.scala b/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/ConcurrentQueryBenchmark.scala
index d8a2176..35ef7f9 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/ConcurrentQueryBenchmark.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/ConcurrentQueryBenchmark.scala
@@ -216,8 +216,6 @@
         .format("carbondata")
         .option("tableName", tableName)
         .option("tempCSV", "false")
-        .option("single_pass", "true")
-        .option("dictionary_exclude", "id") // id is high cardinality column
         .option("table_blocksize", "32")
         .mode(SaveMode.Overwrite)
         .save()
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/SimpleQueryBenchmark.scala b/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/SimpleQueryBenchmark.scala
index 595af20..0d0846c 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/SimpleQueryBenchmark.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/benchmark/SimpleQueryBenchmark.scala
@@ -220,7 +220,6 @@
       input.write
           .format("carbondata")
           .option("tableName", tableName)
-          .option("single_pass", "true")
           .option("table_blocksize", "32")
           .mode(SaveMode.Overwrite)
           .save()
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
deleted file mode 100644
index 808e319..0000000
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.examples
-
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.examples.util.{AllDictionaryUtil, ExampleUtils}
-
-
-object AllDictionaryExample {
-
-  def main(args: Array[String]) {
-    val spark = ExampleUtils.createCarbonSession("AllDictionaryExample")
-    exampleBody(spark)
-    spark.close()
-  }
-
-  def exampleBody(spark : SparkSession): Unit = {
-    val testData = ExampleUtils.currentPath + "/src/main/resources/dataSample.csv"
-    val csvHeader = "ID,date,country,name,phonetype,serialname,salary"
-    val dictCol = "|date|country|name|phonetype|serialname|"
-    val allDictFile = ExampleUtils.currentPath + "/target/data.dictionary"
-
-    // extract all dictionary files from source data
-    AllDictionaryUtil.extractDictionary(spark.sparkContext,
-      testData, allDictFile, csvHeader, dictCol)
-    // Specify date format based on raw data
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
-
-    spark.sql("DROP TABLE IF EXISTS dictionary_table")
-
-    spark.sql(
-      s"""
-         | CREATE TABLE IF NOT EXISTS dictionary_table(
-         | ID Int,
-         | date Date,
-         | country String,
-         | name String,
-         | phonetype String,
-         | serialname String,
-         | salary Int,
-         | floatField float
-         | ) STORED BY 'carbondata'
-       """.stripMargin)
-
-    spark.sql(s"""
-           LOAD DATA LOCAL INPATH '$testData' into table dictionary_table
-           options('ALL_DICTIONARY_PATH'='$allDictFile', 'SINGLE_PASS'='true')
-           """)
-
-    spark.sql("""
-           SELECT * FROM dictionary_table
-           """).show()
-
-    spark.sql("""
-           SELECT * FROM dictionary_table where floatField=3.5
-           """).show()
-
-    CarbonProperties.getInstance().addProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
-    spark.sql("DROP TABLE IF EXISTS dictionary_table")
-
-    // clean local dictionary files
-    AllDictionaryUtil.cleanDictionary(allDictFile)
-  }
-}
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala
index f757bee..a52a016 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlluxioExample.scala
@@ -91,7 +91,6 @@
          | STORED BY 'carbondata'
          | TBLPROPERTIES(
          |    'SORT_COLUMNS' = 'phonetype,name',
-         |    'DICTIONARY_INCLUDE'='phonetype',
          |    'TABLE_BLOCKSIZE'='32',
          |    'AUTO_LOAD_MERGE'='true')
        """.stripMargin)
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
index edb806a..d2b4c95 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
@@ -55,7 +55,6 @@
          | complexData ARRAY<STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
 
     // Alter table change data type
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameComplexTypeExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameComplexTypeExample.scala
index 0abf5c5..663e247 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameComplexTypeExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/DataFrameComplexTypeExample.scala
@@ -46,34 +46,17 @@
   }
 
   def exampleBody(spark: SparkSession): Unit = {
-    val complexTypeDictionaryTableName = s"complex_type_dictionary_table"
     val complexTypeNoDictionaryTableName = s"complex_type_noDictionary_table"
     val complexTypeNoDictionaryTableNameArray = s"complex_type_noDictionary_array_table"
 
     import spark.implicits._
 
     // drop table if exists previously
-    spark.sql(s"DROP TABLE IF EXISTS ${ complexTypeDictionaryTableName }")
     spark.sql(s"DROP TABLE IF EXISTS ${ complexTypeNoDictionaryTableName }")
     spark.sql(s"DROP TABLE IF EXISTS ${ complexTypeNoDictionaryTableNameArray }")
 
     spark.sql(
       s"""
-         | CREATE TABLE ${ complexTypeDictionaryTableName }(
-         | id INT,
-         | name STRING,
-         | city STRING,
-         | salary FLOAT,
-         | file struct<school:array<string>, age:int>
-         | )
-         | STORED BY 'carbondata'
-         | TBLPROPERTIES(
-         | 'sort_columns'='name',
-         | 'dictionary_include'='city')
-         | """.stripMargin)
-
-    spark.sql(
-      s"""
          | CREATE TABLE ${ complexTypeNoDictionaryTableNameArray }(
          | id INT,
          | name STRING,
@@ -83,8 +66,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(
-         | 'sort_columns'='name',
-         | 'dictionary_include'='city')
+         | 'sort_columns'='name')
          | """.stripMargin)
 
 
@@ -104,15 +86,6 @@
 
 
     val sc = spark.sparkContext
-    // generate data
-    val df = sc.parallelize(Seq(
-      ComplexTypeData(1, "index_1", "city_1", 10000.0f,
-        StructElement(Array("struct_11", "struct_12"), 10)),
-      ComplexTypeData(2, "index_2", "city_2", 20000.0f,
-        StructElement(Array("struct_21", "struct_22"), 20)),
-      ComplexTypeData(3, "index_3", "city_3", 30000.0f,
-        StructElement(Array("struct_31", "struct_32"), 30))
-    )).toDF
 
     // generate data
     val df2 = sc.parallelize(Seq(
@@ -131,14 +104,6 @@
         StructElement1(Array("struct_31", "struct_32"), Array("struct_11", "struct_12"), 30))
     )).toDF
 
-
-    df.printSchema()
-    df.write
-      .format("carbondata")
-      .option("tableName", complexTypeDictionaryTableName)
-      .mode(SaveMode.Append)
-      .save()
-
     df1.printSchema()
     df1.write
       .format("carbondata")
@@ -146,7 +111,6 @@
       .mode(SaveMode.Append)
       .save()
 
-
     df2.printSchema()
     df2.write
       .format("carbondata")
@@ -154,28 +118,6 @@
       .mode(SaveMode.Append)
       .save()
 
-
-    spark.sql(s"select count(*) from ${ complexTypeDictionaryTableName }")
-      .show(100, truncate = false)
-
-    spark.sql(s"select * from ${ complexTypeDictionaryTableName } order by id desc")
-      .show(300, truncate = false)
-
-    spark.sql(s"select * " +
-              s"from ${ complexTypeDictionaryTableName } " +
-              s"where id = 100000001 or id = 1 limit 100").show(100, truncate = false)
-
-    spark.sql(s"select * " +
-              s"from ${ complexTypeDictionaryTableName } " +
-              s"where id > 10 limit 100").show(100, truncate = false)
-
-    // show segments
-    spark.sql(s"SHOW SEGMENTS FOR TABLE ${ complexTypeDictionaryTableName }").show(false)
-
-    // drop table
-    spark.sql(s"DROP TABLE IF EXISTS ${ complexTypeDictionaryTableName }")
-
-
     spark.sql(s"select count(*) from ${ complexTypeNoDictionaryTableName }")
       .show(100, truncate = false)
 
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala
index e946fe2..4df859d 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala
@@ -82,8 +82,7 @@
       .sql(
         s"""CREATE TABLE TEST_BOUNDARY (c1_int int,c2_Bigint Bigint,c3_Decimal Decimal(38,30),
            |c4_double double,c5_string string,c6_Timestamp Timestamp,c7_Datatype_Desc string)
-           |STORED BY 'org.apache.carbondata.format' TBLPROPERTIES
-           |('DICTIONARY_INCLUDE'='c6_Timestamp')""".stripMargin)
+           |STORED BY 'org.apache.carbondata.format' """.stripMargin)
 
     inputPath = FileFactory
       .getUpdatedFilePath(s"$rootPath/examples/spark2/src/main/resources/Test_Data1.csv")
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala
index b37fba8..3b44d0e 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala
@@ -69,7 +69,6 @@
          | )
          | STORED BY 'carbondata'
          | LOCATION '$rootPath/examples/spark2/target/store'
-         | TBLPROPERTIES('SORT_COLUMNS'='', 'DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
 
     spark.sql(
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala
index 6c5bfc6..98f02e6 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala
@@ -78,7 +78,6 @@
          | )
          | STORED BY 'carbondata'
          | LOCATION '${args(2)}'
-         | TBLPROPERTIES('SORT_COLUMNS'='', 'DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
 
     spark.sql(
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
index 66d4b71..27a0134 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
@@ -79,8 +79,7 @@
          | charField CHAR(5)
          | )
          | USING carbondata
-         | OPTIONS('DICTIONARY_INCLUDE'='dateField, charField',
-         | 'dbName'='default', 'tableName'='sparksession_table')
+         | OPTIONS('tableName'='sparksession_table')
        """.stripMargin)
 
     val path = s"$rootPath/examples/spark2/src/main/resources/data.csv"
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkStreamingExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkStreamingExample.scala
index beaeee1..d66fff4 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkStreamingExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkStreamingExample.scala
@@ -69,8 +69,7 @@
            | STORED BY 'carbondata'
            | TBLPROPERTIES(
            | 'streaming'='true',
-           | 'sort_columns'='name',
-           | 'dictionary_include'='city')
+           | 'sort_columns'='name')
            | """.stripMargin)
       val carbonTable = CarbonEnv.getCarbonTable(Some("default"), streamTableName)(spark)
       // batch load
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingUsingBatchLoadExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingUsingBatchLoadExample.scala
index 57d4596..3354a77 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingUsingBatchLoadExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingUsingBatchLoadExample.scala
@@ -66,7 +66,6 @@
            | STORED BY 'carbondata'
            | TBLPROPERTIES(
            | 'sort_columns'='name',
-           | 'dictionary_include'='city',
            | 'AUTO_LOAD_MERGE'='true',
            | 'COMPACTION_LEVEL_THRESHOLD'='4,10')
            | """.stripMargin)
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingWithRowParserExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingWithRowParserExample.scala
index ceb3d0f..07c5a8b 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingWithRowParserExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StreamingWithRowParserExample.scala
@@ -59,7 +59,7 @@
              | )
              | STORED BY 'carbondata'
              | TBLPROPERTIES(
-             | 'streaming'='true', 'sort_columns'='name', 'dictionary_include'='city')
+             | 'streaming'='true', 'sort_columns'='name')
              | """.stripMargin)
       } else {
         spark.sql(
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala
index 4e099af..cecc767 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala
@@ -57,7 +57,7 @@
              | )
              | STORED BY 'carbondata'
              | TBLPROPERTIES(
-             | 'streaming'='true', 'sort_columns'='name', 'dictionary_include'='city')
+             | 'streaming'='true', 'sort_columns'='name')
              | """.stripMargin)
       } else {
         spark.sql(
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala
deleted file mode 100644
index 5ce4c98d..0000000
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.examples.util
-
-import java.io.DataOutputStream
-
-import scala.collection.mutable.{ArrayBuffer, HashSet}
-
-import org.apache.spark.SparkContext
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
-
-
-object AllDictionaryUtil {
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-  def extractDictionary(sc: SparkContext,
-                        srcData: String,
-                        outputPath: String,
-                        fileHeader: String,
-                        dictCol: String): Unit = {
-    val fileHeaderArr = fileHeader.split(",")
-    val isDictCol = new Array[Boolean](fileHeaderArr.length)
-    for (i <- 0 until fileHeaderArr.length) {
-      if (dictCol.contains("|" + fileHeaderArr(i).toLowerCase() + "|")) {
-        isDictCol(i) = true
-      } else {
-        isDictCol(i) = false
-      }
-    }
-    val dictionaryRdd = sc.textFile(srcData).flatMap(x => {
-      val tokens = x.split(",")
-      val result = new ArrayBuffer[(Int, String)]()
-      for (i <- 0 until isDictCol.length) {
-        if (isDictCol(i)) {
-          try {
-            result += ((i, tokens(i)))
-          } catch {
-            case _: ArrayIndexOutOfBoundsException =>
-              LOGGER.error("Read a bad record: " + x)
-          }
-        }
-      }
-      result
-    }).groupByKey().flatMap(x => {
-      val distinctValues = new HashSet[(Int, String)]()
-      for (value <- x._2) {
-        distinctValues.add(x._1, value)
-      }
-      distinctValues
-    })
-    val dictionaryValues = dictionaryRdd.map(x => x._1 + "," + x._2).collect()
-    saveToFile(dictionaryValues, outputPath)
-  }
-
-  def cleanDictionary(outputPath: String): Unit = {
-    try {
-      val file = FileFactory.getCarbonFile(outputPath)
-      if (file.exists()) {
-        file.delete()
-      }
-    } catch {
-      case ex: Exception =>
-        LOGGER.error("Clean dictionary catching exception:" + ex)
-    }
-  }
-
-  def saveToFile(contents: Array[String], outputPath: String): Unit = {
-    var writer: DataOutputStream = null
-    try {
-      val file = FileFactory.getCarbonFile(outputPath)
-      if (!file.exists()) {
-        file.createNewFile()
-      }
-      writer = FileFactory.getDataOutputStream(outputPath)
-      for (content <- contents) {
-        writer.writeBytes(content + "\n")
-      }
-    } catch {
-      case ex: Exception =>
-        LOGGER.error("Save dictionary to file catching exception:" + ex)
-    } finally {
-      if (writer != null) {
-        try {
-          writer.close()
-        } catch {
-          case ex: Exception =>
-            LOGGER.error("Close output stream catching exception:" + ex)
-        }
-      }
-    }
-  }
-}
diff --git a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
index b42fd13..caffee6 100644
--- a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
+++ b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
@@ -50,10 +50,6 @@
 
   }
 
-  test("AllDictionaryExample") {
-    AllDictionaryExample.exampleBody(spark)
-  }
-
   test("AlterTableExample") {
     AlterTableExample.exampleBody(spark)
   }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
index f59fe04..a70f4f1 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputFormat.java
@@ -93,11 +93,6 @@
   public static final String BAD_RECORD_PATH = "mapreduce.carbontable.bad.record.path";
   public static final String DATE_FORMAT = "mapreduce.carbontable.date.format";
   public static final String TIMESTAMP_FORMAT = "mapreduce.carbontable.timestamp.format";
-  public static final String IS_ONE_PASS_LOAD = "mapreduce.carbontable.one.pass.load";
-  public static final String DICTIONARY_SERVER_HOST =
-      "mapreduce.carbontable.dict.server.host";
-  public static final String DICTIONARY_SERVER_PORT =
-      "mapreduce.carbontable.dict.server.port";
   /**
    * Set the update timestamp if user sets in case of update query. It needs to be updated
    * in load status update time
@@ -405,12 +400,6 @@
       }
     }
     model.setBadRecordsLocation(badRecordsPath);
-    model.setUseOnePass(
-        conf.getBoolean(IS_ONE_PASS_LOAD,
-            Boolean.parseBoolean(
-                carbonProperty.getProperty(
-                    CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-                    CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT))));
     return model;
   }
 
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 c63aa68..aece1ef 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
@@ -24,7 +24,6 @@
 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.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;
@@ -60,11 +59,9 @@
         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(
             carbonTable.getAbsoluteTableIdentifier(),
-            carbonColumns[i].getColumnIdentifier(), dataTypes[i], dictionaryPath));
+            carbonColumns[i].getColumnIdentifier(), dataTypes[i]));
       } else {
         dataTypes[i] = carbonColumns[i].getDataType();
       }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java
index e4819ee..b2ac9d1 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/CarbonStreamInputFormat.java
@@ -23,7 +23,6 @@
 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;
@@ -138,11 +137,9 @@
             CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DICTIONARY);
         Dictionary dictionary = null;
         if (isDictionary) {
-          String dictionaryPath = carbontable.getTableInfo().getFactTable().getTableProperties()
-              .get(CarbonCommonConstants.DICTIONARY_PATH);
           DictionaryColumnUniqueIdentifier dictionarIdentifier =
               new DictionaryColumnUniqueIdentifier(carbontable.getAbsoluteTableIdentifier(),
-                  child.getColumnIdentifier(), child.getDataType(), dictionaryPath);
+                  child.getColumnIdentifier(), child.getDataType());
           dictionary = cache.get(dictionarIdentifier);
         }
         queryType =
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
index 064f6bb..4b5248f 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
@@ -17,30 +17,20 @@
 
 package org.apache.carbondata.hadoop.testutil;
 
-import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.DataOutputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
-import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.nio.charset.Charset;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.UUID;
 
 import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-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.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
@@ -51,7 +41,6 @@
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -61,7 +50,6 @@
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
@@ -69,13 +57,7 @@
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
 import org.apache.carbondata.core.writer.ThriftWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator;
 import org.apache.carbondata.processing.loading.DataLoadExecutor;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 import org.apache.carbondata.processing.loading.csvinput.BlockDetails;
@@ -107,14 +89,9 @@
   private AbsoluteTableIdentifier absoluteTableIdentifier;
   private String storePath = null;
   private String csvPath;
-  private boolean dictionary;
   private List<String> sortColumns = new ArrayList<>();
 
   public StoreCreator(String storePath, String csvPath) {
-    this(storePath, csvPath, false);
-  }
-
-  public StoreCreator(String storePath, String csvPath, boolean dictionary) {
     this.storePath = storePath;
     this.csvPath = csvPath;
     String dbName = "testdb";
@@ -126,7 +103,6 @@
     sortColumns.add("serialname");
     absoluteTableIdentifier = AbsoluteTableIdentifier.from(storePath + "/testdb/testtable",
         new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
-    this.dictionary = dictionary;
   }
 
   public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
@@ -209,7 +185,6 @@
     }
 
     CarbonTable table = createTable(absoluteTableIdentifier);
-    writeDictionary(csvPath, table);
     return buildCarbonLoadModel(table, csvPath, absoluteTableIdentifier);
   }
 
@@ -225,9 +200,6 @@
     tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
     List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
     ArrayList<Encoding> encodings = new ArrayList<>();
-    if (dictionary) {
-      encodings.add(Encoding.DICTIONARY);
-    }
     int schemaOrdinal = 0;
     ColumnSchema id = new ColumnSchema();
     id.setColumnName("id");
@@ -379,58 +351,6 @@
     return newColumnSchema;
   }
 
-  private void writeDictionary(String factFilePath, CarbonTable table) throws Exception {
-    BufferedReader reader = new BufferedReader(new InputStreamReader(
-        new FileInputStream(factFilePath), "UTF-8"));
-    List<CarbonDimension> dims = table.getVisibleDimensions();
-    Set<String>[] set = new HashSet[dims.size()];
-    for (int i = 0; i < set.length; i++) {
-      set[i] = new HashSet<String>();
-    }
-    String line = reader.readLine();
-    while (line != null) {
-      String[] data = line.split(",");
-      for (int i = 0; i < set.length; i++) {
-        set[i].add(data[i]);
-      }
-      line = reader.readLine();
-    }
-
-    Cache dictCache = CacheProvider.getInstance()
-        .createCache(CacheType.REVERSE_DICTIONARY);
-    for (int i = 0; i < set.length; i++) {
-      ColumnIdentifier columnIdentifier =
-          new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          new DictionaryColumnUniqueIdentifier(
-              table.getAbsoluteTableIdentifier(), columnIdentifier, columnIdentifier.getDataType());
-      CarbonDictionaryWriter writer =
-          new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-      for (String value : set[i]) {
-        writer.write(value);
-      }
-      writer.close();
-      writer.commit();
-      Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-              columnIdentifier, dims.get(i).getDataType()));
-      CarbonDictionarySortInfoPreparator preparator =
-          new CarbonDictionarySortInfoPreparator();
-      List<String> newDistinctValues = new ArrayList<String>();
-      CarbonDictionarySortInfo dictionarySortInfo =
-          preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
-      CarbonDictionarySortIndexWriter carbonDictionaryWriter =
-          new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
-      try {
-        carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
-        carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
-      } finally {
-        carbonDictionaryWriter.close();
-      }
-    }
-    reader.close();
-  }
-
   public void setSortColumns(List<String> sortColumns) {
     this.sortColumns = sortColumns;
   }
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
index bdcdcf7..3a29d58 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
@@ -29,6 +29,7 @@
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
@@ -126,8 +127,8 @@
   }
 
   @Test public void testInputFormatMapperReadAllRowsAndColumns() throws Exception {
+    String outPath = "target/output";
     try {
-      String outPath = "target/output";
       CarbonProjection carbonProjection = new CarbonProjection();
       carbonProjection.addColumn("ID");
       carbonProjection.addColumn("date");
@@ -145,6 +146,7 @@
       throw e;
     } finally {
       creator.clearDataMaps();
+      FileFactory.deleteAllFilesOfDir(new File(outPath));
     }
   }
 
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 def8b26..2dd053a 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
@@ -29,7 +29,6 @@
 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.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;
@@ -37,7 +36,6 @@
 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;
-import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -89,11 +87,9 @@
         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(carbonTable.getAbsoluteTableIdentifier(),
-                carbonColumns[i].getColumnIdentifier(), dataTypes[i], dictionaryPath));
+                carbonColumns[i].getColumnIdentifier(), dataTypes[i]));
       } else {
         dataTypes[i] = carbonColumns[i].getDataType();
       }
@@ -268,12 +264,6 @@
     if (obj == null) {
       return null;
     }
-    if (carbonColumn.hasEncoding(Encoding.DICTIONARY)) {
-      obj = DataTypeUtil.getDataBasedOnDataType(obj.toString(), dataType);
-      if (obj == null) {
-        return null;
-      }
-    }
     if (dataType == DataTypes.NULL) {
       return null;
     } else if (dataType == DataTypes.DOUBLE) {
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonObjectInspector.java b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonObjectInspector.java
index 4f82b76..160a5ac 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonObjectInspector.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/CarbonObjectInspector.java
@@ -60,14 +60,16 @@
   }
 
   private ObjectInspector getObjectInspector(final TypeInfo typeInfo) {
-    if (typeInfo.equals(TypeInfoFactory.doubleTypeInfo)) {
+    if (typeInfo.equals(TypeInfoFactory.stringTypeInfo)) {
+      return PrimitiveObjectInspectorFactory.writableStringObjectInspector;
+    } else if (typeInfo.equals(TypeInfoFactory.doubleTypeInfo)) {
       return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+    } else if (typeInfo.equals(TypeInfoFactory.floatTypeInfo)) {
+      return PrimitiveObjectInspectorFactory.writableFloatObjectInspector;
     } else if (typeInfo.equals(TypeInfoFactory.intTypeInfo)) {
       return PrimitiveObjectInspectorFactory.writableIntObjectInspector;
     } else if (typeInfo.equals(TypeInfoFactory.longTypeInfo)) {
       return PrimitiveObjectInspectorFactory.writableLongObjectInspector;
-    } else if (typeInfo.equals(TypeInfoFactory.stringTypeInfo)) {
-      return PrimitiveObjectInspectorFactory.writableStringObjectInspector;
     } else if (typeInfo instanceof DecimalTypeInfo) {
       return PrimitiveObjectInspectorFactory
           .getPrimitiveWritableObjectInspector((DecimalTypeInfo) typeInfo);
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 4bbd931..40da535 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
@@ -14,18 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.carbondata.presto
 
-import java.util.Optional
-
-import com.facebook.presto.spi.block.{Block, VariableWidthBlock}
-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.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
-import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.cache.dictionary.Dictionary
+import org.apache.carbondata.core.metadata.datatype.DataType
 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
@@ -51,70 +44,12 @@
     dataTypes = new Array[DataType](carbonColumns.length)
 
     carbonColumns.zipWithIndex.foreach {
-      case (carbonColumn, index) => if (carbonColumn.hasEncoding(Encoding.DICTIONARY) &&
-                                        !carbonColumn.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-                                        !carbonColumn.isComplex) {
-        val cacheProvider: CacheProvider = CacheProvider.getInstance
-        val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
-          cacheProvider
-            .createCache(CacheType.FORWARD_DICTIONARY)
+      case (carbonColumn, index) =>
         dataTypes(index) = carbonColumn.getDataType
-        val dictionaryPath: String = carbonTable.getTableInfo.getFactTable.getTableProperties
-          .get(CarbonCommonConstants.DICTIONARY_PATH)
-        dictionaries(index) = forwardDictionaryCache
-          .get(new DictionaryColumnUniqueIdentifier(carbonTable.getAbsoluteTableIdentifier,
-            carbonColumn.getColumnIdentifier, dataTypes(index), dictionaryPath))
-      } else {
-        dataTypes(index) = carbonColumn.getDataType
-      }
     }
 
   }
 
-  /**
-   * Function to create the SliceArrayBlock with dictionary Data
-   *
-   * @param dictionaryData
-   * @return
-   */
-  private def createDictionaryBlock(dictionaryData: Dictionary): Block = {
-    val chunks: DictionaryChunksWrapper = dictionaryData.getDictionaryChunks
-    val positionCount = chunks.getSize
-
-   // In dictionary there will be only one null and the key value will be 1 by default in carbon,
-   // hence the isNullVector will be populated only once with null value it has no bearing on
-   // actual data.
-
-    val offsetVector : Array[Int] = new Array[Int](positionCount + 2 )
-    val isNullVector: Array[Boolean] = new Array[Boolean](positionCount + 1)
-    // the first value is just a filler as we always start with index 1 in carbon
-    isNullVector(0) = true
-    isNullVector(1) = true
-    var count = 0
-    var byteArray = new Array[Byte](0)
-    // The Carbondata key starts from 1 so we need a filler at 0th position hence adding filler to
-    // offset, hence 0th Position -> 0
-    offsetVector(0) = 0
-    while (chunks.hasNext) {
-      val value: Array[Byte] = chunks.next
-      if (count == 0) {
-        // 1 index is actually Null to map to carbondata null values .
-        // 1st Position -> 0 (For actual Null)
-        offsetVector(count + 1) = 0
-        // 2nd Postion -> 0 as the byte[] is still null so starting point will be 0 only
-        offsetVector(count + 2) = 0
-      } else {
-        byteArray = byteArray ++ value
-        offsetVector(count + 2) = byteArray.length
-      }
-      count += 1
-    }
-    new VariableWidthBlock(positionCount + 1,
-      wrappedBuffer(byteArray, 0, byteArray.length),
-      offsetVector,
-      Optional.ofNullable(isNullVector))
-  }
-
   override def readRow(data: Array[AnyRef]): T = {
     throw new RuntimeException("UnSupported Method")
   }
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeLocalDictTest.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeLocalDictTest.scala
index e9bde45..e901113 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeLocalDictTest.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeLocalDictTest.scala
@@ -110,7 +110,7 @@
       Map("NAME" -> "sahil"),
       Map("NAME" -> null)
     )
-    assert(actualResult.equals(expectedResult))
+    assertResult(expectedResult)(actualResult)
   }
 
   test("test and filter clause with greater than expression") {
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
index 966791e..21c95de 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
@@ -506,19 +506,6 @@
       .executeQuery("SELECT id AS RESULT FROM TESTDB.TESTTABLE WHERE isCurrentEmployee is NOT null AND ID>8")
     assert(actualResult.head("RESULT").toString.toInt==9)
   }
-  test("test the is null operator when null is included in string data type dictionary_include"){
-    // See CARBONDATA-2155
-    val actualResult: List[Map[String, Any]] = prestoServer.executeQuery("SELECT SERIALNAME  FROM TESTDB.TESTTABLE WHERE SERIALNAME IS NULL")
-    assert(actualResult equals List(Map("SERIALNAME" -> null)))
-  }
-  test("test the min function when null is included in string data type with dictionary_include"){
-    // See CARBONDATA-2152
-    val actualResult = prestoServer.executeQuery("SELECT MIN(SERIALNAME) FROM TESTDB.TESTTABLE")
-    val expectedResult = List(Map("_col0" -> "ASD14875"))
-
-    assert(actualResult.equals(expectedResult))
-  }
-
 
   test("test the show schemas result"){
    val actualResult = prestoServer.executeQuery("SHOW SCHEMAS")
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 8a48c89..8f1ef0a 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
@@ -21,7 +21,8 @@
 import java.nio.charset.Charset
 import java.text.SimpleDateFormat
 import java.util
-import java.util.{ArrayList, Date, List, UUID}
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.{ArrayList, Date, UUID}
 
 import scala.collection.JavaConversions._
 import scala.collection.mutable
@@ -35,24 +36,20 @@
 import org.apache.hadoop.mapreduce.{RecordReader, TaskType}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier, ReverseDictionary}
-import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.fileoperations.{AtomicFileOperationFactory, AtomicFileOperations, FileWriteOperation}
 import org.apache.carbondata.core.metadata.converter.{SchemaConverter, ThriftWrapperSchemaConverterImpl}
-import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.datatype.{DataTypes, StructField}
 import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension, CarbonMeasure, ColumnSchema}
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
-import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, CarbonTableBuilder, TableSchemaBuilder}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter, CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
-import org.apache.carbondata.core.writer.{CarbonDictionaryWriter, CarbonDictionaryWriterImpl, ThriftWriter}
+import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.processing.loading.DataLoadExecutor
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
 import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, CSVRecordReaderIterator, StringArrayWritable}
@@ -78,9 +75,7 @@
         new CarbonTableIdentifier(dbName,
           tableName,
           UUID.randomUUID().toString))
-      val storeDir: File = new File(absoluteTableIdentifier.getTablePath)
       val table: CarbonTable = createTable(absoluteTableIdentifier, useLocalDict)
-      writeDictionary(dataFilePath, table, absoluteTableIdentifier)
       val schema: CarbonDataLoadSchema = new CarbonDataLoadSchema(table)
       val loadModel: CarbonLoadModel = new CarbonLoadModel()
       import scala.collection.JavaConverters._
@@ -135,172 +130,40 @@
       loadModel.setMaxColumns("15")
       executeGraph(loadModel, storePath)
     } catch {
-      case e: Exception => e.printStackTrace()
-
+      case e: Exception =>
+        throw e
     }
   }
 
   private def createTable(absoluteTableIdentifier: AbsoluteTableIdentifier,
       useLocalDict: Boolean): CarbonTable = {
-    val tableInfo: TableInfo = new TableInfo()
-    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath)
-    tableInfo.setDatabaseName(
-      absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName)
-    val tableSchema: TableSchema = new TableSchema()
-    tableSchema.setTableName(
-      absoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
-    val columnSchemas = new ArrayList[ColumnSchema]()
-    val dictionaryEncoding: ArrayList[Encoding] = new ArrayList[Encoding]()
-    if (!useLocalDict) {
-      dictionaryEncoding.add(Encoding.DICTIONARY)
-    }
 
-    val invertedIndexEncoding: ArrayList[Encoding] = new ArrayList[Encoding]()
-    invertedIndexEncoding.add(Encoding.INVERTED_INDEX)
+    val integer = new AtomicInteger(0)
+    val schemaBuilder = new TableSchemaBuilder
+    schemaBuilder.addColumn(new StructField("ID", DataTypes.INT), integer, false, false)
+    schemaBuilder.addColumn(new StructField("date", DataTypes.DATE), integer, false, false)
+    schemaBuilder.addColumn(new StructField("country", DataTypes.STRING), integer, false, false)
+    schemaBuilder.addColumn(new StructField("name", DataTypes.STRING), integer, false, false)
+    schemaBuilder.addColumn(new StructField("phonetype", DataTypes.STRING), integer, false, false)
+    schemaBuilder.addColumn(new StructField("serialname", DataTypes.STRING), integer, false, false)
+    schemaBuilder.addColumn(new StructField("salary", DataTypes.DOUBLE), integer, false, false)
+    schemaBuilder.addColumn(new StructField("bonus", DataTypes.createDecimalType(10, 4)), integer, false, true)
+    schemaBuilder.addColumn(new StructField("monthlyBonus", DataTypes.createDecimalType(18, 4)), integer, false, true)
+    schemaBuilder.addColumn(new StructField("dob", DataTypes.TIMESTAMP), integer, false, true)
+    schemaBuilder.addColumn(new StructField("shortField", DataTypes.SHORT), integer, false, false)
+    schemaBuilder.addColumn(new StructField("isCurrentEmployee", DataTypes.BOOLEAN), integer, false, true)
+    schemaBuilder.tableName(absoluteTableIdentifier.getTableName)
+    val schema = schemaBuilder.build()
 
-    val id: ColumnSchema = new ColumnSchema()
-    id.setColumnName("ID")
-    id.setDataType(DataTypes.INT)
-    id.setEncodingList(dictionaryEncoding)
-    id.setColumnUniqueId(UUID.randomUUID().toString)
-    id.setColumnReferenceId(id.getColumnUniqueId)
-    id.setDimensionColumn(true)
-    id.setSchemaOrdinal(0)
-    columnSchemas.add(id)
+    val builder = new CarbonTableBuilder
+    builder.databaseName(absoluteTableIdentifier.getDatabaseName)
+      .tableName(absoluteTableIdentifier.getTableName)
+      .tablePath(absoluteTableIdentifier.getTablePath)
+      .isTransactionalTable(true)
+      .tableSchema(schema)
+    val carbonTable = builder.build()
 
-    val directDictionaryEncoding: util.ArrayList[Encoding] = new util.ArrayList[Encoding]()
-    directDictionaryEncoding.add(Encoding.DIRECT_DICTIONARY)
-    directDictionaryEncoding.add(Encoding.DICTIONARY)
-    directDictionaryEncoding.add(Encoding.INVERTED_INDEX)
-
-    val date: ColumnSchema = new ColumnSchema()
-    date.setColumnName("date")
-    date.setDataType(DataTypes.DATE)
-    date.setEncodingList(directDictionaryEncoding)
-    date.setColumnUniqueId(UUID.randomUUID().toString)
-    date.setDimensionColumn(true)
-    date.setColumnReferenceId(date.getColumnUniqueId)
-    date.setSchemaOrdinal(1)
-    columnSchemas.add(date)
-
-    val country: ColumnSchema = new ColumnSchema()
-    country.setColumnName("country")
-    country.setDataType(DataTypes.STRING)
-    country.setEncodingList(dictionaryEncoding)
-    country.setColumnUniqueId(UUID.randomUUID().toString)
-    country.setColumnReferenceId(country.getColumnUniqueId)
-    country.setDimensionColumn(true)
-    country.setSchemaOrdinal(2)
-    country.setColumnReferenceId(country.getColumnUniqueId)
-    columnSchemas.add(country)
-
-    val name: ColumnSchema = new ColumnSchema()
-    name.setColumnName("name")
-    name.setDataType(DataTypes.STRING)
-    name.setEncodingList(dictionaryEncoding)
-    name.setColumnUniqueId(UUID.randomUUID().toString)
-    name.setDimensionColumn(true)
-    name.setSchemaOrdinal(3)
-    name.setColumnReferenceId(name.getColumnUniqueId)
-    columnSchemas.add(name)
-
-    val phonetype: ColumnSchema = new ColumnSchema()
-    phonetype.setColumnName("phonetype")
-    phonetype.setDataType(DataTypes.STRING)
-    phonetype.setEncodingList(dictionaryEncoding)
-    phonetype.setColumnUniqueId(UUID.randomUUID().toString)
-    phonetype.setDimensionColumn(true)
-    phonetype.setSchemaOrdinal(4)
-    phonetype.setColumnReferenceId(phonetype.getColumnUniqueId)
-    columnSchemas.add(phonetype)
-
-    val serialname: ColumnSchema = new ColumnSchema()
-    serialname.setColumnName("serialname")
-    serialname.setDataType(DataTypes.STRING)
-    serialname.setEncodingList(dictionaryEncoding)
-    serialname.setColumnUniqueId(UUID.randomUUID().toString)
-    serialname.setDimensionColumn(true)
-    serialname.setSchemaOrdinal(5)
-    serialname.setColumnReferenceId(serialname.getColumnUniqueId)
-    columnSchemas.add(serialname)
-
-    val salary: ColumnSchema = new ColumnSchema()
-    salary.setColumnName("salary")
-    salary.setDataType(DataTypes.DOUBLE)
-    salary.setEncodingList(new util.ArrayList[Encoding]())
-    salary.setColumnUniqueId(UUID.randomUUID().toString)
-    salary.setDimensionColumn(false)
-    salary.setSchemaOrdinal(6)
-    salary.setColumnReferenceId(salary.getColumnUniqueId)
-    columnSchemas.add(salary)
-
-    val bonus: ColumnSchema = new ColumnSchema()
-    bonus.setColumnName("bonus")
-    bonus.setDataType(DataTypes.createDecimalType(10, 4))
-    bonus.setPrecision(10)
-    bonus.setScale(4)
-    bonus.setEncodingList(dictionaryEncoding)
-    bonus.setEncodingList(invertedIndexEncoding)
-    bonus.setColumnUniqueId(UUID.randomUUID().toString)
-    bonus.setDimensionColumn(false)
-    bonus.setSchemaOrdinal(7)
-    bonus.setColumnReferenceId(bonus.getColumnUniqueId)
-    columnSchemas.add(bonus)
-
-    val monthlyBonus: ColumnSchema = new ColumnSchema()
-    monthlyBonus.setColumnName("monthlyBonus")
-    monthlyBonus.setDataType(DataTypes.createDecimalType(18, 4))
-    monthlyBonus.setPrecision(18)
-    monthlyBonus.setScale(4)
-    monthlyBonus.setSchemaOrdinal(8)
-    monthlyBonus.setEncodingList(invertedIndexEncoding)
-    monthlyBonus.setColumnUniqueId(UUID.randomUUID().toString)
-    monthlyBonus.setDimensionColumn(false)
-    monthlyBonus.setColumnReferenceId(monthlyBonus.getColumnUniqueId)
-    columnSchemas.add(monthlyBonus)
-
-    val dob: ColumnSchema = new ColumnSchema()
-    dob.setColumnName("dob")
-    dob.setDataType(DataTypes.TIMESTAMP)
-    dob.setEncodingList(directDictionaryEncoding)
-    dob.setColumnUniqueId(UUID.randomUUID().toString)
-    dob.setDimensionColumn(true)
-    dob.setSchemaOrdinal(9)
-    dob.setColumnReferenceId(dob.getColumnUniqueId)
-    columnSchemas.add(dob)
-
-    val shortField: ColumnSchema = new ColumnSchema()
-    shortField.setColumnName("shortField")
-    shortField.setDataType(DataTypes.SHORT)
-    shortField.setEncodingList(dictionaryEncoding)
-    shortField.setColumnUniqueId(UUID.randomUUID().toString)
-    shortField.setDimensionColumn(true)
-    shortField.setSchemaOrdinal(10)
-    shortField.setColumnReferenceId(shortField.getColumnUniqueId)
-    columnSchemas.add(shortField)
-
-    val isCurrentEmployee: ColumnSchema = new ColumnSchema()
-    isCurrentEmployee.setColumnName("isCurrentEmployee")
-    isCurrentEmployee.setDataType(DataTypes.BOOLEAN)
-    isCurrentEmployee.setEncodingList(invertedIndexEncoding)
-    isCurrentEmployee.setColumnUniqueId(UUID.randomUUID().toString)
-    isCurrentEmployee.setDimensionColumn(false)
-    isCurrentEmployee.setColumnReferenceId(isCurrentEmployee.getColumnUniqueId)
-    columnSchemas.add(isCurrentEmployee)
-
-    tableSchema.setListOfColumns(columnSchemas)
-    val schemaEvol: SchemaEvolution = new SchemaEvolution()
-    schemaEvol.setSchemaEvolutionEntryList(
-      new util.ArrayList[SchemaEvolutionEntry]())
-    tableSchema.setSchemaEvolution(schemaEvol)
-    tableSchema.setTableId(UUID.randomUUID().toString)
-    tableSchema.getTableProperties.put(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE,
-      String.valueOf(useLocalDict))
-    tableInfo.setTableUniqueName(
-      absoluteTableIdentifier.getCarbonTableIdentifier.getTableUniqueName
-    )
-    tableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    tableInfo.setFactTable(tableSchema)
+    val tableInfo = carbonTable.getTableInfo
     val schemaFilePath: String = CarbonTablePath.getSchemaFilePath(
       absoluteTableIdentifier.getTablePath)
     val schemaMetadataPath: String =
@@ -328,85 +191,6 @@
     CarbonMetadata.getInstance.getCarbonTable(tableInfo.getTableUniqueName)
   }
 
-  private def writeDictionary(factFilePath: String,
-      table: CarbonTable,
-      absoluteTableIdentifier: AbsoluteTableIdentifier): Unit = {
-    val reader: BufferedReader = new BufferedReader(
-      new FileReader(factFilePath))
-    val header: String = reader.readLine()
-    val allCols: util.List[CarbonColumn] = new util.ArrayList[CarbonColumn]()
-    val dimensions: util.List[CarbonDimension] = table.getVisibleDimensions
-    allCols.addAll(dimensions)
-    val msrs: List[CarbonMeasure] = table.getVisibleMeasures
-    allCols.addAll(msrs)
-    val dimensionsIndex = dimensions.map(dim => dim.getColumnSchema.getSchemaOrdinal)
-    val dimensionSet: Array[util.List[String]] = Array.ofDim[util.List[String]](dimensions.size)
-
-    for (i <- dimensionSet.indices) {
-      dimensionSet(i) = new util.ArrayList[String]()
-    }
-    var line: String = reader.readLine()
-    while (line != null) {
-      val data: Array[String] = line.split(",")
-      for (index <- dimensionSet.indices) {
-        addDictionaryValuesToDimensionSet(dimensions, dimensionsIndex, dimensionSet, data, index)
-      }
-      line = reader.readLine()
-    }
-    val dictCache: Cache[DictionaryColumnUniqueIdentifier, ReverseDictionary] = CacheProvider
-      .getInstance.createCache(CacheType.REVERSE_DICTIONARY)
-
-    for (index <- dimensionSet.indices) {
-      val columnIdentifier: ColumnIdentifier =
-        new ColumnIdentifier(dimensions.get(index).getColumnId, null, null)
-
-      val dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(
-          table.getAbsoluteTableIdentifier,
-          columnIdentifier,
-          columnIdentifier.getDataType)
-      val writer: CarbonDictionaryWriter = new CarbonDictionaryWriterImpl(
-        dictionaryColumnUniqueIdentifier)
-      for (value <- dimensionSet(index).distinct) {
-        writer.write(value)
-      }
-      writer.close()
-      writer.commit()
-      val dict: Dictionary = dictCache
-        .get(
-          new DictionaryColumnUniqueIdentifier(
-            absoluteTableIdentifier,
-            columnIdentifier,
-            dimensions.get(index).getDataType)
-        )
-        .asInstanceOf[Dictionary]
-      val preparator: CarbonDictionarySortInfoPreparator =
-        new CarbonDictionarySortInfoPreparator()
-      val newDistinctValues: List[String] = new ArrayList[String]()
-      val dictionarySortInfo: CarbonDictionarySortInfo =
-        preparator.getDictionarySortInfo(newDistinctValues,
-          dict,
-          dimensions.get(index).getDataType)
-      val carbonDictionaryWriter: CarbonDictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier)
-      try {
-        carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex)
-        carbonDictionaryWriter.writeInvertedSortIndex(
-          dictionarySortInfo.getSortIndexInverted)
-      }
-      catch {
-        case exception: Exception =>
-
-
-          logger.error(s"exception occurs $exception")
-          throw new CarbonDataLoadingException("Data Loading Failed")
-      }
-      finally carbonDictionaryWriter.close()
-    }
-    reader.close()
-  }
-
-
   private def addDictionaryValuesToDimensionSet(dims: util.List[CarbonDimension],
       dimensionIndex: mutable.Buffer[Int],
       dimensionSet: Array[util.List[String]],
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/AlterTableTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/AlterTableTestCase.scala
index cc34df5..4b7bde6 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/AlterTableTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/AlterTableTestCase.scala
@@ -150,7 +150,7 @@
   //Check delete column for dimension column
   test("DeleteCol_001_01", Include) {
     intercept[Exception] {
-      sql(s"""create table test1 (name string, id int) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='id') """).collect
+      sql(s"""create table test1 (name string, id int) stored by 'carbondata'  """).collect
       sql(s"""insert into test1 select 'xx',1""").collect
       sql(s"""alter table test1 drop columns (name)""").collect
       sql(s"""select name from test1""").collect
@@ -186,7 +186,7 @@
   //Check delete column for multiple column
   test("DeleteCol_001_04", Include) {
     intercept[Exception] {
-      sql(s"""create table test1 (name string, country string, upd_time timestamp, id int) stored by 'carbondata'  TBLPROPERTIES('DICTIONARY_INCLUDE'='id')""").collect
+      sql(s"""create table test1 (name string, country string, upd_time timestamp, id int) stored by 'carbondata' """).collect
       sql(s"""insert into test1 select 'xx','yy',current_timestamp,1""").collect
       sql(s"""alter table test1 drop columns (name, upd_time)""").collect
       sql(s"""select name, upd_time from test1""").collect
@@ -207,7 +207,7 @@
   //Check delete column for include dictionary column
   test("DeleteCol_001_06", Include) {
     intercept[Exception] {
-      sql(s"""create table test1 (name string, id int) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='id')""").collect
+      sql(s"""create table test1 (name string, id int) stored by 'carbondata' """).collect
       sql(s"""insert into test1 select 'xx',1""").collect
       sql(s"""alter table test1 drop columns (id)""").collect
       sql(s"""select id from test1""").collect
@@ -323,7 +323,7 @@
    sql(s"""create table test1 (name string, id int) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx',1""").collect
    sql(s"""insert into test1 select 'xx',11""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (id1 int) tblproperties('dictionary_include'='id1')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (id1 int) """).collect
     checkAnswer(s"""select id1 from test1""",
       Seq(Row(null), Row(null)), "AlterTableTestCase_AddColumn_001_04")
      sql(s"""drop table if exists test1""").collect
@@ -336,7 +336,7 @@
      sql(s"""create table test1 (name string, id int) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx',1""").collect
    sql(s"""insert into test1 select 'xx',11""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (price decimal(10,6)) TBLPROPERTIES('DEFAULT.VALUE.price'='11.111', 'dictionary_include'='price')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (price decimal(10,6)) TBLPROPERTIES('DEFAULT.VALUE.price'='11.111')""").collect
     checkAnswer(s"""select sum(price) from test1 where price = 11.111""",
       Seq(Row(22.222000)), "AlterTableTestCase_AddColumn_001_05")
      sql(s"""drop table if exists test1""").collect
@@ -348,7 +348,7 @@
     sql(s"""drop table if exists test1""").collect
      sql(s"""create table test1 (name string, id int) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx',1""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (price bigint) TBLPROPERTIES('DEFAULT.VALUE.Price'='1.1','dictionary_include'='price')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (price bigint) TBLPROPERTIES('DEFAULT.VALUE.Price'='1.1')""").collect
     checkAnswer(s"""select price from test1""",
       Seq(Row(null)), "AlterTableTestCase_AddColumn_001_06")
      sql(s"""drop table if exists test1""").collect
@@ -360,7 +360,7 @@
     sql(s"""drop table if exists test1""").collect
      sql(s"""create table test1 (name string, id int) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx',1""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (price bigint) TBLPROPERTIES('DEFAULT.VALUE.Price'='11','dictionary_include'='price')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (price bigint) TBLPROPERTIES('DEFAULT.VALUE.Price'='11')""").collect
     checkAnswer(s"""select count(id) from test1 where price = 11""",
       Seq(Row(1)), "AlterTableTestCase_AddColumn_001_07")
      sql(s"""drop table if exists test1""").collect
@@ -372,7 +372,7 @@
      sql(s"""drop table if exists test1""").collect
    sql(s"""create table test1 (name string, id int) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx',1""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (id1 int, country string) tblproperties('dictionary_include'='id1')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (id1 int, country string) """).collect
     checkAnswer(s"""select id1, country from test1""",
       Seq(Row(null,null)), "AlterTableTestCase_AddColumn_001_08")
      sql(s"""drop table if exists test1""").collect
@@ -384,7 +384,7 @@
      sql(s"""drop table if exists test1""").collect
    sql(s"""create table test1 (name string) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx'""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (Id int) TBLPROPERTIES('DICTIONARY_INCLUDE'='id')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (Id int)  """).collect
     checkAnswer(s"""select id from test1""",
       Seq(Row(null)), "AlterTableTestCase_AddColumn_001_09")
      sql(s"""drop table if exists test1""").collect
@@ -396,7 +396,7 @@
     sql(s"""drop table if exists test1""").collect
      sql(s"""create table test1 (name string) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx'""").collect
-   sql(s"""ALTER TABLE test1 ADD COLUMNS (upd_time timestamp, country string) TBLPROPERTIES('DICTIONARY_EXCLUDE'='country')""").collect
+   sql(s"""ALTER TABLE test1 ADD COLUMNS (upd_time timestamp, country string) """).collect
     checkAnswer(s"""select country, upd_time from test1""",
       Seq(Row(null,null)), "AlterTableTestCase_AddColumn_001_10")
      sql(s"""drop table if exists test1""").collect
@@ -421,7 +421,7 @@
       sql(s"""drop table if exists test1""").collect
       sql(s"""create table test1 (name string) stored by 'carbondata'""").collect
       sql(s"""insert into test1 select 'xx'""").collect
-      sql(s"""ALTER TABLE test1 ADD COLUMNS (Id int) TBLPROPERTIES('DICTIONARY_INCLUDE'='id','default.value.name'='yy')""").collect
+      sql(s"""ALTER TABLE test1 ADD COLUMNS (Id int) TBLPROPERTIES('default.value.name'='yy')""").collect
     }
     sql(s"""drop table if exists test1""").collect
   }
@@ -487,7 +487,7 @@
   //check drop table when table is altered by adding columns
   test("DropTable_001_05", Include) {
      sql(s"""drop table if exists test1""").collect
-   sql(s"""create table test1 (name string, id int) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='id')""").collect
+   sql(s"""create table test1 (name string, id int) stored by 'carbondata'  """).collect
    sql(s"""insert into test1 select 'xx',1""").collect
    sql(s"""ALTER TABLE test1 ADD COLUMNS (upd_time timestamp, country string) TBLPROPERTIES( 'DEFAULT.VALUE.country'='China')""").collect
    sql(s"""insert into test1 select 'yy',1,current_timestamp,'xx'""").collect
@@ -501,7 +501,7 @@
     sql(s"""drop table if exists test1""").collect
      sql(s"""create table test1 (country string, name string) stored by 'carbondata' """).collect
    sql(s"""insert into test1 select 'xx','uu'""").collect
-    sql(s"""alter table test1 add columns (price decimal(10,4)) tblproperties('dictionary_include'='price','DEFAULT.VALUE.price'='11.111')""").collect
+    sql(s"""alter table test1 add columns (price decimal(10,4)) tblproperties('DEFAULT.VALUE.price'='11.111')""").collect
      sql(s"""drop table if exists test1""").collect
   }
 
@@ -511,7 +511,7 @@
     sql(s"""drop table if exists test1""").collect
      sql(s"""create table test1 (name string, id decimal(3,2),country string) stored by 'carbondata' """).collect
    sql(s"""insert into test1 select 'xx',1.22,'china'""").collect
-   sql(s"""alter table test1 add columns (price decimal(10,4)) tblproperties('dictionary_include'='price','DEFAULT.VALUE.price'='11.111')""").collect
+   sql(s"""alter table test1 add columns (price decimal(10,4)) tblproperties('DEFAULT.VALUE.price'='11.111')""").collect
     checkAnswer(s"""select * from test1""",
       Seq(Row("xx",1.22,"china",11.1110)), "AlterTableTestCase_Dictionary_001_01")
      sql(s"""drop table if exists test1""").collect
@@ -872,7 +872,7 @@
      sql(s"""create table test1(name string) stored by 'carbondata'""").collect
    sql(s"""insert into test1 select 'xx1'""").collect
    sql(s"""insert into test1 select 'xx2'""").collect
-   sql(s"""alter table test1 add columns (id int) tblproperties('dictionary_include'='id')""").collect
+   sql(s"""alter table test1 add columns (id int)  """).collect
    sql(s"""insert into test1 select 'xx1',1""").collect
    sql(s"""alter table test1 add columns (country string)""").collect
    sql(s"""insert into test1 select 'xx1',1, 'china'""").collect
@@ -971,7 +971,7 @@
   test("AlterTable-001-AltersameTablename-001-TC001", Include) {
      sql(s"""drop table  if exists uniqdata""").collect
    sql(s"""drop table  if exists uniqdata1""").collect
-   sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,INTEGER_COLUMN1,CUST_ID')""").collect
+   sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
    sql(s"""alter table uniqdata RENAME TO  uniqdata1""").collect
    sql(s"""alter table uniqdata1 RENAME TO uniqdata""").collect
@@ -984,7 +984,7 @@
 
   //Check select query after alter the int to Bigint and decimal Lower Precision to higher precision
   test("AlterTable-007-selectquery-001-TC002", Include) {
-     sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,INTEGER_COLUMN1,CUST_ID')""").collect
+     sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/2000_UniqData.csv' into table uniqdata1 OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
    sql(s"""ALTER TABLE uniqdata1 CHANGE CUST_ID CUST_ID BIGINT""").collect
     sql(s"""select * from uniqdata1 where cust_name like 'Cust%'""").collect
@@ -995,7 +995,7 @@
 
   //Check select query after alter from lower to higher precision
   test("AlterTable-008-selectquery-001-TC003", Include) {
-     sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,INTEGER_COLUMN1,CUST_ID')""").collect
+     sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/2000_UniqData.csv' into table uniqdata1 OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
    sql(s"""ALTER TABLE uniqdata1 CHANGE decimal_column1 decimal_column1 DECIMAL(31,11)""").collect
     sql(s"""select * from uniqdata1 where cust_name like 'Cust%'""").collect
@@ -1007,8 +1007,8 @@
   //Check add column on Decimal,Timestamp,int,string,Bigint
   test("AlterTable-002-001-TC-004", Include) {
      sql(s"""drop table if exists uniqdata59""").collect
-   sql(s"""CREATE TABLE uniqdata59 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,INTEGER_COLUMN1,CUST_ID')""").collect
-    sql(s"""ALTER TABLE uniqdata59 ADD COLUMNS (a1 int,a2 int,a3 decimal,a4 Bigint,a5 String,a6 timestamp,a7 Bigint,a8 decimal(10,2),a9 timestamp,a10 String,a11 string,a12 string,a13 string,a14 string,a15 string,a16 string,a17 string,a18 string,a19 string,a20 string,a21 string,a22 string,a23 string,a24 string,a25 string,a26 string,a27 string,a28 string,a29 string,a30 string,a31 string,a32 string,a33 string,a34 string,a35 string,a36 string,a37 string,a38 string,a39 string,a40 string,a41 string,a42 string,a43 string,a44 string,a45 string,a46 string,a47 string,a48 string,a49 string,a50 string,a51 string,a52 string,a53 string,a54 string,a55 string,a56 string,a57 string,a58 string,a59 string,a60 string,a61 string,a62 string,a63 string,a64 string,a65 string,a66 string,a67 string,a68 string,a69 string,a70 string,a71 string,a72 string,a73 string,a74 string,a75 string,a76 string,a77 string,a78 string,a79 string,a80 string,a81 string,a82 string,a83 string,a84 string,a85 string,a86 string,a87 string,a88 string) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')""").collect
+   sql(s"""CREATE TABLE uniqdata59 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
+    sql(s"""ALTER TABLE uniqdata59 ADD COLUMNS (a1 int,a2 int,a3 decimal,a4 Bigint,a5 String,a6 timestamp,a7 Bigint,a8 decimal(10,2),a9 timestamp,a10 String,a11 string,a12 string,a13 string,a14 string,a15 string,a16 string,a17 string,a18 string,a19 string,a20 string,a21 string,a22 string,a23 string,a24 string,a25 string,a26 string,a27 string,a28 string,a29 string,a30 string,a31 string,a32 string,a33 string,a34 string,a35 string,a36 string,a37 string,a38 string,a39 string,a40 string,a41 string,a42 string,a43 string,a44 string,a45 string,a46 string,a47 string,a48 string,a49 string,a50 string,a51 string,a52 string,a53 string,a54 string,a55 string,a56 string,a57 string,a58 string,a59 string,a60 string,a61 string,a62 string,a63 string,a64 string,a65 string,a66 string,a67 string,a68 string,a69 string,a70 string,a71 string,a72 string,a73 string,a74 string,a75 string,a76 string,a77 string,a78 string,a79 string,a80 string,a81 string,a82 string,a83 string,a84 string,a85 string,a86 string,a87 string,a88 string) """).collect
      sql(s"""drop table  if exists uniqdata59""").collect
   }
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BadRecordTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BadRecordTestCase.scala
index a4cd333..84d6304 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BadRecordTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BadRecordTestCase.scala
@@ -143,14 +143,14 @@
 
   //Show segments for table when data loading having parameters BAD_RECORDS_ACTION=FAIL/FORCE/REDIRECT/IGNORE,BAD_RECORD_LOGGER_ENABLE=true/false and IS_EMPTY_DATA_BAD_RECORD=false/true
   test("BadRecords-001_PTS020_TC001", Include) {
-     sql(s"""CREATE TABLE badrecordTest13 (ID int,CUST_ID int,cust_name string) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID')""").collect
+     sql(s"""CREATE TABLE badrecordTest13 (ID int,CUST_ID int,cust_name string) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/badrecord/test3.csv' into table badrecordTest13 OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='ID,CUST_ID,cust_name')""").collect
     sql(s"""SHOW SEGMENTS FOR TABLE badrecordTest13""").collect
      sql(s"""drop table if exists badrecordTest13""").collect
   }
 
 
-  //Create table and Load data with parameters BAD_RECORDS_ACTION=FAIL/FORCE/REDIRECT/IGNORE,BAD_RECORD_LOGGER_ENABLE=true/false and IS_EMPTY_DATA_BAD_RECORD=false/true  for date and char types using single pass and vectorized reader parameters
+  //Create table and Load data with parameters BAD_RECORDS_ACTION=FAIL/FORCE/REDIRECT/IGNORE,BAD_RECORD_LOGGER_ENABLE=true/false and IS_EMPTY_DATA_BAD_RECORD=false/true  for date and char types using vectorized reader parameters
   test("BadRecords-001_PTS012_TC001", Include) {
      sql(s"""CREATE TABLE badrecordtest14 (ID int,CUST_ID int,cust_name string) STORED BY 'org.apache.carbondata.format'""").collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/badrecord/test3.csv' into table badrecordtest14 OPTIONS('FILEHEADER'='ID,CUST_ID,cust_name','DELIMITER'=',' , 'QUOTECHAR'='"','is_empty_data_bad_record'='false','BAD_RECORDS_ACTION'='IGNORE')""").collect
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
index 077e007..f03d0cf 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BloomFilterDataMapTestCase.scala
@@ -70,7 +70,6 @@
          | TBLPROPERTIES(
          |  'LONG_STRING_COLUMNS'='longStringField',
          |  'SORT_COLUMNS'='stringSortField',
-         |  'DICTIONARY_INCLUDE'='stringDictField',
          |  'local_dictionary_enable'='true',
          |  'local_dictionary_threshold'='10000',
          |  'local_dictionary_include'='stringLocalDictField',
@@ -152,7 +151,6 @@
          | STORED BY 'carbondata'
          | TBLPROPERTIES(
          |  'LONG_STRING_COLUMNS'='longStringField',
-         |  'DICTIONARY_INCLUDE'='stringDictField',
          |  'local_dictionary_enable'='true',
          |  'local_dictionary_threshold'='10000',
          |  'local_dictionary_include'='stringLocalDictField',
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala
index 32fcc70..92b94d0 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala
@@ -57,20 +57,6 @@
     }
   }
 
-  test("test Int column as bucketcolumns through dictionary_include") {
-    sql("DROP TABLE IF EXISTS bucket_table")
-    sql("CREATE TABLE bucket_table (ID Int, date Timestamp, country String, name String, phonetype String," +
-        "serialname String, salary Int) STORED BY 'carbondata' TBLPROPERTIES " +
-        "('DICTIONARY_INCLUDE'='ID','BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='ID')")
-    sql(s"LOAD DATA INPATH '$resourcesPath/source.csv' INTO TABLE bucket_table")
-    val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default_bucket_table")
-    if (table != null && table.getBucketingInfo != null) {
-      assert(true)
-    } else {
-      assert(false, "Bucketing info does not exist")
-    }
-  }
-
   test("test multi columns as bucketcolumns") {
     sql("DROP TABLE IF EXISTS bucket_table")
     sql("CREATE TABLE bucket_table (ID Int, date Timestamp, country String, name String, phonetype String," +
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ColumndictTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ColumndictTestCase.scala
deleted file mode 100644
index 54740c6..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ColumndictTestCase.scala
+++ /dev/null
@@ -1,492 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import org.apache.spark.sql.common.util._
-import org.scalatest.BeforeAndAfterAll
-
-/**
- * Test Class for columndictTestCase to verify all scenerios
- */
-
-class ColumndictTestCase extends QueryTest with BeforeAndAfterAll {
-         
-
-  //Load history data from CSV with/without header and specify/don't specify headers in command using external ALL_dictionary_PATH
-  test("Columndict-TC001", Include) {
-     sql(s"""drop table if exists t3""").collect
-   sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load history data from CSV with/without header and specify/don't specify headers in command using external columndict
-  test("Columndict-TC002", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external All_dictionary_path for CSV having incomplete/wrong data/no data/null data
-  test("Columndict-TC003", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/inValidData.dictionary', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external columndict for CSV having incomplete/wrong data/no data/null data
-  test("Columndict-TC004", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/inValidData.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load multiple CSV from folder into table , Multiple level of folders using external all_dictionary_path
-  test("Columndict-TC005", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data1' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load multiple CSV from folder into table , Multiple level of folders using external columndict
-  ignore("Columndict-TC006", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data1' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using CSV file with different extension (.dat, .xls, .doc,.txt) and without extension from external dictionary
-  ignore("Columndict-TC007", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.dat' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using CSV file with different extension (.dat, .xls, .doc,.txt) and without extension from external dictionary
-  ignore("Columndict-TC008", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.dat' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using MAXCOLUMNS during loading with external all_dictionary_path
-  test("Columndict-TC009", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.dat' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary','maxcolumns'='8', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using MAXCOLUMNS during loading with external columndict
-  ignore("Columndict-TC010", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.dat' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv','maxcolumns'='8', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Bad records logging after load using external all_dictionary_path
-  ignore("Columndict-TC011", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary','BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Bad records logging after load using external columndict
-  ignore("Columndict-TC012", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'=
-  'country:$resourcesPath/Data/columndict/country.csv','BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Incremental Load using external dictionary
-  test("Columndict-TC013", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select * from t3 where ID>=5""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Incremental Load using external dictionary
-  ignore("Columndict-TC014", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select * from t3 where ID>=5""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external dictionary for table without table properties
-  ignore("Columndict-TC015", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external dictionary for table without table properties
-  ignore("Columndict-TC016", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external all_dictionary_path for table with table properties(DICTIONARY_EXCLUDE, DICTIONARY_INCLUDE, BLOCKSIZE)
-  ignore("Columndict-TC017", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES ('TABLE_BLOCKSIZE'= '256 MB','DICTIONARY_INCLUDE'='salary','DICTIONARY_EXCLUDE'='phonetype')""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external columndict for table with table properties(DICTIONARY_EXCLUDE, DICTIONARY_INCLUDE, BLOCKSIZE)
-  test("Columndict-TC018", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES ('TABLE_BLOCKSIZE'= '256 MB','DICTIONARY_INCLUDE'='salary','DICTIONARY_EXCLUDE'='phonetype')""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external all_dictionary_path for measure and table properties(DICTIONARY_EXCLUDE, DICTIONARY_INCLUDE, BLOCKSIZE)
-  ignore("Columndict-TC019", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES ('TABLE_BLOCKSIZE'= '256 MB','DICTIONARY_INCLUDE'='salary','DICTIONARY_EXCLUDE'='country')""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='salary:$resourcesPath/Data/columndict/salary.csv', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Load using external columndict for table with measure and tableproperties(DICTIONARY_EXCLUDE, DICTIONARY_INCLUDE, BLOCKSIZE)
-  test("Columndict-TC020", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES ('TABLE_BLOCKSIZE'= '256 MB','DICTIONARY_EXCLUDE'='country')""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:'resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Columndict parameter name validation
-  ignore("Columndict-TC021", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES ('TABLE_BLOCKSIZE'= '256 MB','DICTIONARY_EXCLUDE'='country')""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='countries:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Columndict parameter value validation
-  test("Columndict-TC022", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='salary:$resourcesPath/Data/columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for data validation in csv(empty/null/wrong data) for all_dictionary_path
-  ignore("Columndict-TC023", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/inValidData.dictionary', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for data validation in csv(empty/null/wrong data) for columndict
-  test("Columndict-TC024", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('COLUMNDICT'='country:'resourcesPath/Data/columndict/inValidData.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for validation of external all_dictionary_path folder with incorrect path
-  test("Columndict-TC025", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('ALL_DICTIONARY_PATH'=''resourcesPath/Data/*.dictionary', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for validation of external all_dictionary_path folder with correct path
-  test("Columndict-TC026", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/*.dictionary', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for validation of external columndict folder with correct path
-  test("Columndict-TC027", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('COLUMNDICT'='country:'resourcesPath/Data/columndict/*.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for validation of external all_dictionary_path file( missing /wrong path / wrong name)
-  test("Columndict-TC028", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('ALL_DICTIONARY_PATH'=''resourcesPath/Data/columndict/wrongName.dictionary', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for validation of external columndict file( missing /wrong path / wrong name)
-  test("Columndict-TC029", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('COLUMNDICT'='country:'resourcesPath/Data/columndict/wrongName.csv', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for different dictionary file extensions for all_dictionary_path
-  test("Columndict-TC030", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.txt', 'SINGLE_PASS'='true')""").collect
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //Check for different dictionary file extensions for columndict
-  test("Columndict-TC031", Include) {
-    intercept[Exception] {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-      sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/inValidData.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data/columndict/country.txt', 'SINGLE_PASS'='true')""").collect
-    }
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check limit for all_dictionary_path
-  test("Columndict-TC032", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 limit 100""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check count for all_dictionary_path
-  ignore("Columndict-TC033", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select count(*) from t3""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check sum for all_dictionary_path
-  ignore("Columndict-TC034", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select sum(salary) from t3""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check >= for all_dictionary_path
-  test("Columndict-TC035", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where ID >=5""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check != for all_dictionary_path
-  ignore("Columndict-TC036", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where ID != 9""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check between for all_dictionary_path
-  ignore("Columndict-TC037", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id between 2 and 9""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check like for all_dictionary_path
-  ignore("Columndict-TC038", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id Like '9%'""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check group by for all_dictionary_path
-  ignore("Columndict-TC039", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id > 3 group by id,name having id = 2""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check sort by for all_dictionary_path
-  ignore("Columndict-TC040", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id > 4 sort by name desc""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check limit for columndict
-  ignore("Columndict-TC041", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 limit 100""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check count for columndict
-  ignore("Columndict-TC042", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select count(*) from t3""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check sum for columndict
-  ignore("Columndict-TC043", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select sum(salary) from t3""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check >= for columndict
-  ignore("Columndict-TC044", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where ID >=5""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check != for columndict
-  ignore("Columndict-TC045", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where ID != 9""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check between for columndict
-  ignore("Columndict-TC046", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id between 2 and 9""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check like for columndict
-  ignore("Columndict-TC047", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id Like '9%'""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check group by for columndict
-  ignore("Columndict-TC048", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id > 3 group by id,name having id = 2""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-
-  //To check sort by for columndict
-  ignore("Columndict-TC049", Include) {
-     sql(s"""CREATE TABLE IF NOT EXISTS t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata'""").collect
-   sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/columndict/data.csv' into table t3 options('COLUMNDICT'='country:$resourcesPath/Data//columndict/country.csv', 'SINGLE_PASS'='true')""").collect
-    sql(s"""select ID,name from t3 where id > 4 sort by name desc""").collect
-
-     sql(s"""drop table if exists t3""").collect
-  }
-
-}
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ComplexDataTypeTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ComplexDataTypeTestCase.scala
index 8d6333c..723ecf5 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ComplexDataTypeTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/ComplexDataTypeTestCase.scala
@@ -129,7 +129,7 @@
         109.0, 2738.562)))
   }
 
-  // check create table with complex data type with dictionary_include columns and insert into
+  // check create table with complex data type and insert into
   // into complex table
   test("test Complex_DataType-003") {
     CarbonProperties.getInstance()
@@ -138,8 +138,7 @@
     sql("DROP TABLE IF EXISTS test")
     sql(
       "create table test(person struct<detail:struct<id:int,name:string,height:double," +
-      "status:boolean,dob:date,dobt:timestamp>>) stored by 'carbondata' tblproperties" +
-      "('dictionary_include'='person')")
+      "status:boolean,dob:date,dobt:timestamp>>) stored by 'carbondata' ")
     sql("insert into test values(named_struct('detail', named_struct('id', 1, 'name', 'abc', 'height', 4.30, 'status', true, 'dob', '2017-08-09', 'dobt', '2017-08-09 00:00:00.0')))")
     checkAnswer(sql("select * from test"),
       Seq(Row(Row(Row(1,
@@ -148,8 +147,7 @@
     sql("DROP TABLE IF EXISTS test")
     sql(
       "create table test(p1 array<int>,p2 array<string>,p3 array<double>,p4 array<boolean>,p5 " +
-      "array<date>,p6 array<timestamp>) stored by 'carbondata' tblproperties" +
-      "('dictionary_include'='p1,p2,p3,p4,p5,p6')")
+      "array<date>,p6 array<timestamp>) stored by 'carbondata' ")
     sql("insert into test values(array(1,2,3), array('abc','def','mno'), array(4.30,4.60,5.20), array(true,true,false), array('2017-08-09','2017-08-09','2017-07-07'), array('2017-08-09 00:00:00.0','2017-08-09 00:00:00.0','2017-07-07 00:00:00.0'))")
     checkAnswer(sql("select * from test"),
       Seq(Row(mutable.WrappedArray.make(Array(1, 2, 3)),
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CreateTableWithLocalDictionaryTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CreateTableWithLocalDictionaryTestCase.scala
index ec9cc8d..70325ff 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CreateTableWithLocalDictionaryTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/CreateTableWithLocalDictionaryTestCase.scala
@@ -121,7 +121,7 @@
         """
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name','local_dictionary_include'='name')
+          | tblproperties('local_dictionary_enable'='true','local_dictionary_include'='name')
         """.
           stripMargin)
     }
@@ -352,8 +352,7 @@
         """
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='20000','local_dictionary_include'='name',
-          | 'dictionary_include'='name')
+          | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='20000','local_dictionary_include'='name')
         """.stripMargin)
     }
   }
@@ -396,7 +395,7 @@
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
           | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='23213497321591234324',
-          | 'local_dictionary_include'='name','dictionary_include'='name')
+          | 'local_dictionary_include'='name')
         """.stripMargin)
     }
   }
@@ -512,7 +511,7 @@
         """
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
+          | tblproperties('local_dictionary_enable'='true',
           | 'local_dictionary_include'='name')
         """.
           stripMargin)
@@ -611,7 +610,7 @@
         """
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
+          | tblproperties('local_dictionary_enable'='true',
           | 'local_dictionary_exclude'='name')
         """.
           stripMargin)
@@ -704,7 +703,7 @@
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
           | tblproperties('local_dictionary_exclude'='name','local_dictionary_include'='city',
-          | 'local_dictionary_enable'='true','dictionary_include'='name,city')
+          | 'local_dictionary_enable'='true')
         """.
           stripMargin)
     }
@@ -1014,7 +1013,7 @@
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
           | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='20000',
-          | 'local_dictionary_include'='name','dictionary_include'='name')
+          | 'local_dictionary_include'='name' )
         """.stripMargin)
     }
   }
@@ -1062,8 +1061,7 @@
           | CREATE TABLE local1(id int, name string, city string, age int)
           | STORED BY 'org.apache.carbondata.format'
           | tblproperties('local_dictionary_enable'='true',
-          | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name',
-          | 'dictionary_include'='name')
+          | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name')
         """.stripMargin)
     }
   }
@@ -1165,7 +1163,7 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('local_dictionary_enable'='false','dictionary_include'='name',
+        | tblproperties('local_dictionary_enable'='false' ,
         | 'local_dictionary_include'='name')
       """.
         stripMargin)
@@ -1397,7 +1395,7 @@
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
         | tblproperties('local_dictionary_enable'='false','local_dictionary_threshold'='20000',
-        | 'local_dictionary_include'='name','dictionary_include'='name')
+        | 'local_dictionary_include'='name' )
       """.stripMargin)
 
     val descLoc = sql("describe formatted local1").collect
@@ -1454,8 +1452,7 @@
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
         | tblproperties('local_dictionary_enable'='false',
-        | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name',
-        | 'dictionary_include'='name')
+        | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name')
       """.stripMargin)
 
     val descLoc = sql("describe formatted local1").collect
@@ -1471,7 +1468,8 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='global_sort',
+        | tblproperties(
+        | 'sort_scope'='global_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1491,7 +1489,7 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='local_sort',
+        | tblproperties('sort_scope'='local_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1510,7 +1508,8 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='no_sort',
+        | tblproperties(
+        | 'sort_scope'='no_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1529,7 +1528,8 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='local_sort',
+        | tblproperties(
+        | 'sort_scope'='local_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1696,7 +1696,7 @@
       sql(
         """
           | CREATE TABLE local1 STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
+          | tblproperties('local_dictionary_enable'='true' ,
           | 'local_dictionary_include'='name') as select * from local
         """.
           stripMargin)
@@ -1828,7 +1828,7 @@
       sql(
         """
           | CREATE TABLE local1 STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
+          | tblproperties('local_dictionary_enable'='true' ,
           | 'local_dictionary_exclude'='name') as select * from local
         """.
           stripMargin)
@@ -1910,7 +1910,7 @@
         """
           | CREATE TABLE local1 STORED BY 'org.apache.carbondata.format'
           | tblproperties('local_dictionary_exclude'='name','local_dictionary_include'='city',
-          | 'local_dictionary_enable'='true','dictionary_include'='name,city') as select * from local
+          | 'local_dictionary_enable'='true') as select * from local
         """.
           stripMargin)
     }
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingIUDTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingIUDTestCase.scala
index c344f81..952b90b 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingIUDTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingIUDTestCase.scala
@@ -901,7 +901,7 @@
 //Check for update Carbon table where a update column is measue and is defined with include ddictionary. Check dictionary file being updated.
 test("IUD-01-01-01_005-12", Include) {
    sql(s"""drop table IF EXISTS default.t_carbn01 """).collect
- sql(s"""create table default.t_carbn01 (Item_type_cd INT, Profit DECIMAL(3,2))STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Item_type_cd')""").collect
+ sql(s"""create table default.t_carbn01 (Item_type_cd INT, Profit DECIMAL(3,2))STORED BY 'org.apache.carbondata.format' """).collect
  sql(s"""insert into default.t_carbn01  select item_type_cd, profit from default.t_carbn01b""").collect
  sql(s"""update default.t_carbn01  set (item_type_cd) = (100100)""").collect
   checkAnswer(s"""select item_type_cd from default.t_carbn01  group by item_type_cd""",
@@ -913,7 +913,7 @@
 //Check for update Carbon table where a update column is dimension and is defined with exclude dictionary. 
 test("IUD-01-01-01_005-13", Include) {
   sql(s"""drop table IF EXISTS default.t_carbn01 """).collect
-  sql(s"""create table default.t_carbn01 (Item_type_cd INT, Profit DECIMAL(3,2))STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Item_type_cd')""").collect
+  sql(s"""create table default.t_carbn01 (Item_type_cd INT, Profit DECIMAL(3,2))STORED BY 'org.apache.carbondata.format' """).collect
   sql(s"""insert into default.t_carbn01  select item_type_cd, profit from default.t_carbn01b""").collect
   val currProperty = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
     .CARBON_BAD_RECORDS_ACTION);
@@ -932,7 +932,7 @@
 
 //Check for update Carbon table where a update column is dimension and is defined with exclude dictionary. 
 test("IUD-01-01-01_005-14", Include) {
-   sql(s"""create table if not exists default.t_carbn01 (Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Item_code')""").collect
+   sql(s"""create table if not exists default.t_carbn01 (Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' """).collect
  sql(s""" insert into default.t_carbn01  select * from default.t_carbn01b""").collect
  sql(s"""update default.t_carbn01  set (Item_code) = ('Ram')""").collect
   checkAnswer(s"""select item_code from default.t_carbn01  group by item_code""",
@@ -943,7 +943,7 @@
 
 //Check for update Carbon table where a update column is dimension and is defined with exclude dictionary. 
 test("IUD-01-01-01_005-15", Include) {
-   sql(s"""create table if not exists default.t_carbn01 (Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Item_code')""").collect
+   sql(s"""create table if not exists default.t_carbn01 (Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' """).collect
  sql(s""" insert into default.t_carbn01  select * from default.t_carbn01b""").collect
  sql(s"""update default.t_carbn01  set (Item_code) = ('123')""").collect
   checkAnswer(s"""select item_code from default.t_carbn01  group by item_code""",
@@ -2582,12 +2582,12 @@
    sql(s"""drop table if exists  t_carbn1""").collect
  sql(s"""drop table if exists  t_carbn2""").collect
  sql(s"""drop table if exists  t_hive01""").collect
- sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn1 select 2, 200000,23.3,'Apple','2012-11-11 11:11:11'""").collect
  sql(s"""insert into t_carbn1 select 2,300000,33.3,'Orange','2012-11-11 11:11:11'""").collect
  sql(s"""insert into t_carbn1 select 2, 200000,23.3,'Banana','2012-11-11 11:11:11'""").collect
  sql(s"""create table t_hive01 as select * from t_carbn1""").collect
- sql(s"""create table t_carbn2(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn2(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn2 select * from t_hive01""").collect
  sql(s"""insert into t_carbn2 select 2, 200000,23.3,'Banana','2012-11-11 11:11:11'""").collect
  sql(s"""delete from t_carbn2 where item_name = 'Banana'""").collect
@@ -2630,10 +2630,10 @@
 test("IUD-01-01-02_023-21", Include) {
    sql(s"""drop table if exists  t_carbn1""").collect
  sql(s"""drop table if exists  t_carbn2""").collect
- sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn1 select 2, 200000,23.3,'Apple','2012-11-11 11:11:11'""").collect
  sql(s"""insert into t_carbn1 select 2,300000,33.3,'Orange','2012-11-11 11:11:11'""").collect
- sql(s"""create table t_carbn2(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn2(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn2 select * from t_carbn1""").collect
  sql(s"""delete from t_carbn2 where item_name = 'Apple'""").collect
  sql(s"""clean files for table t_carbn2""").collect
@@ -2647,10 +2647,10 @@
 test("IUD-01-01-02_023-22", Include) {
    sql(s"""drop table if exists  t_carbn1""").collect
  sql(s"""drop table if exists  t_carbn2""").collect
- sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn1 select 2, 200000,23.3,'Apple','2012-11-11 11:11:11'""").collect
  sql(s"""insert into t_carbn1 select 2,300000,33.3,'Orange','2012-11-11 11:11:11'""").collect
- sql(s"""create table t_carbn2(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn2(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn2 select * from t_carbn1""").collect
  sql(s"""delete from t_carbn2 where item_name in ('Orange','Apple')""").collect
  sql(s"""clean files for table t_carbn2""").collect
@@ -2663,7 +2663,7 @@
 //Test horizontal compaction when different segments same data ,full delete across all segments
 test("IUD-01-01-02_023-23", Include) {
    sql(s"""drop table if exists  t_carbn1""").collect
- sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn1 select 2, 200000,23.3,'Apple','2012-11-11 11:11:11'""").collect
  sql(s"""insert into t_carbn1 select 2,300000,33.3,'Apple','2012-11-11 11:11:11'""").collect
  sql(s"""delete from t_carbn1 where item_name = 'Apple'""").collect
@@ -2820,7 +2820,7 @@
 //Test horizontal compaction when h-compcation threshold set to higher value(10),full updates multiple time
 test("IUD-01-01-02_023-29", Include) {
    sql(s"""drop table if exists  t_carbn1""").collect
- sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='sell_price')""").collect
+ sql(s"""create table t_carbn1(item_type_cd int, sell_price bigint, profit decimal(10,4), item_name string, update_time timestamp) stored by 'carbondata'  """).collect
  sql(s"""insert into t_carbn1 select 2, 200000,23.3,'Apple','2012-11-11 11:11:11'""").collect
  sql(s"""insert into t_carbn1 select 3, 300000,33.3,'Orange','2012-11-11 11:11:11'""").collect
  sql(s"""update t_carbn1 set (sell_price) = (2)""").collect
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingTestCase.scala
index ff8e193..bd1f499 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/DataLoadingTestCase.scala
@@ -62,7 +62,7 @@
 
   //Data load--->Action--->IGNORE--->Logger-->True
   test("BadRecord_Dataload_003", Include) {
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
+     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'  """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='IGNORE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     checkAnswer(s"""select count(*) from uniqdata""",
       Seq(Row(2010)), "DataLoadingTestCase-BadRecord_Dataload_003")
@@ -73,7 +73,7 @@
   //Data load--->Action--->Ignore--->Logger-->False
   test("BadRecord_Dataload_004", Include) {
     sql(s"""drop table if exists uniqdata""").collect
-     sql(s""" CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
+     sql(s""" CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'  """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='IGNORE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     checkAnswer(s"""select count(*) from uniqdata""",
       Seq(Row(2010)), "DataLoadingTestCase-BadRecord_Dataload_004")
@@ -95,24 +95,13 @@
   //Data load--->Action--->Redirect--->Logger-->False
   test("BadRecord_Dataload_006", Include) {
     sql(s"""drop table if exists uniqdata""").collect
-     sql(s""" CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
+     sql(s""" CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'  """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='REDIRECT','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     checkAnswer(s"""select count(*) from uniqdata""",
       Seq(Row(2010)), "DataLoadingTestCase-BadRecord_Dataload_006")
      sql(s"""drop table uniqdata""").collect
   }
 
-
-  //Data load-->Dictionary_Exclude
-  test("BadRecord_Dataload_007", Include) {
-     sql(s"""CREATE TABLE uniq_exclude (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='CUST_NAME,ACTIVE_EMUI_VERSION','DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/2000_UniqData.csv' into table uniq_exclude OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='REDIRECT','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniq_exclude""",
-      Seq(Row(2010)), "DataLoadingTestCase-BadRecord_Dataload_007")
-     sql(s"""drop table uniq_exclude""").collect
-  }
-
-
   //Data load-->Extra_Column_in table
   test("BadRecord_Dataload_010", Include) {
      sql(s"""CREATE TABLE exceed_column_in_table (cust_id int ,CUST_NAME String,date timestamp,date2 timestamp) STORED BY 'org.apache.carbondata.format'""").collect
@@ -601,7 +590,7 @@
   //Check for insert into carbon table with select on Hive table and inserted carbon table created with one dimension excluded from dictionary.
   test("Insert_Func_060", Include) {
      sql(s"""drop table IF EXISTS t_carbn020""").collect
-   sql(s"""create table T_Carbn020(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Item_code')""").collect
+   sql(s"""create table T_Carbn020(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""Insert into T_Carbn020 select * from T_Hive1""").collect
     checkAnswer(s"""select active_status,item_type_cd,qty_day_avg,qty_total,sell_price,sell_pricep,discount_price, profit,item_code,item_name from T_Carbn020 order by update_time""",
       s"""select active_status,item_type_cd,qty_day_avg,qty_total,sell_price,sell_pricep,discount_price, profit,item_code,item_name from T_Hive1 order by update_time""", "DataLoadingTestCase-Insert_Func_060")
@@ -613,7 +602,7 @@
   test("Insert_Func_061", Include) {
      sql(s"""drop table IF EXISTS t_carbn020""").collect
     dropTable("T_Carbn01")
-   sql(s"""create table T_Carbn020(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Item_code')""").collect
+   sql(s"""create table T_Carbn020(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""create table T_Carbn01(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format'""").collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/T_Hive1.csv' INTO table T_Carbn01 options ('DELIMITER'=',', 'QUOTECHAR'='\', 'FILEHEADER'='Active_status,Item_type_cd,Qty_day_avg,Qty_total,Sell_price,Sell_pricep,Discount_price,Profit,Item_code,Item_name,Outlet_name,Update_time,Create_date')""").collect
    sql(s"""Insert into T_Carbn020 select * from T_Carbn01""").collect
@@ -720,7 +709,7 @@
   //Show loads--->Action=Fail--->Logger=True
   test("BadRecord_Dataload_024", Include) {
     dropTable("uniqdata")
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
+     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'  """).collect
     intercept[Exception] {
       sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FAIL','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     }
@@ -890,7 +879,7 @@
 
   //Check for the incremental load data DML without "DELIMITER" specified loading the data successfully.
   test("Incremental_Data_Load_001_001-001-TC-09_840", Include) {
-     sql(s"""create table DL_WithOutDELIMITER(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Update_time')""").collect
+     sql(s"""create table DL_WithOutDELIMITER(Active_status String,Item_type_cd INT,Qty_day_avg INT,Qty_total INT,Sell_price BIGINT,Sell_pricep DOUBLE,Discount_price DOUBLE,Profit DECIMAL(3,2),Item_code String,Item_name String,Outlet_name String,Update_time TIMESTAMP,Create_date String)STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/T_Hive1.csv' INTO table DL_WithOutDELIMITER options ('QUOTECHAR'='\', 'FILEHEADER'='Active_status,Item_type_cd,Qty_day_avg,Qty_total,Sell_price,Sell_pricep,Discount_price,Profit,Item_code,Item_name,Outlet_name,Update_time,Create_date')""").collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/T_Hive1.csv' INTO table DL_WithOutDELIMITER options ('QUOTECHAR'='\', 'FILEHEADER'='Active_status,Item_type_cd,Qty_day_avg,Qty_total,Sell_price,Sell_pricep,Discount_price,Profit,Item_code,Item_name,Outlet_name,Update_time,Create_date')""").collect
     checkAnswer(s"""select count(*) from DL_WithOutDELIMITER""",
@@ -901,7 +890,7 @@
 
   //Check for correct result set displayed for query execution after historical data loading.
   test("History_Data_Load_001_001-002-TC-01_749", Include) {
-     sql(s"""CREATE TABLE  DL_HistoricalData( CUST_ID String,CUST_COUNTRY String,CUST_STATE String,CUST_CITY String,CUST_JOB_TITLE String,CUST_BUY_POTENTIAL String,PROD_UNQ_MDL_ID String,PROD_BRAND_NAME String,PRODUCT_NAME String,PRODUCT_MODEL String,PROD_MODEL_ID String,PROD_COLOR String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,TRACKING_NO String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_NO String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_NICK_NAME String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_PRFRD_FLG String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_LOGIN String,CUST_EMAIL_ADDR String,CUST_LAST_RVW_DATE String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_BAR_CODE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='CUST_STATE','DICTIONARY_INCLUDE'='LATEST_OS_VERSION')""").collect
+     sql(s"""CREATE TABLE  DL_HistoricalData( CUST_ID String,CUST_COUNTRY String,CUST_STATE String,CUST_CITY String,CUST_JOB_TITLE String,CUST_BUY_POTENTIAL String,PROD_UNQ_MDL_ID String,PROD_BRAND_NAME String,PRODUCT_NAME String,PRODUCT_MODEL String,PROD_MODEL_ID String,PROD_COLOR String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,TRACKING_NO String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_NO String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_NICK_NAME String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_PRFRD_FLG String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_LOGIN String,CUST_EMAIL_ADDR String,CUST_LAST_RVW_DATE String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_BAR_CODE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/OSCON.csv' INTO TABLE DL_HistoricalData OPTIONS('FILEHEADER'='CUST_ID,CUST_COUNTRY,CUST_STATE,CUST_CITY,ACTIVE_AREA_ID,ACTIVE_COUNTRY,ACTIVE_PROVINCE,ACTIVE_CITY,ACTIVE_DISTRICT,LATEST_AREAID,LATEST_COUNTRY,LATEST_PROVINCE,LATEST_CITY,LATEST_DISTRICT,WH_COUNTRY,WH_STATE,WH_CITY,WH_COUNTY,CUST_JOB_TITLE,CUST_BUY_POTENTIAL,PROD_UNQ_MDL_ID,PROD_BRAND_NAME,PRODUCT_NAME,PRODUCT_MODEL,PROD_MODEL_ID,PROD_STD_CST,REG_UNIT_PRICE,TOTAL_PRD_COST,TOTAL_PRD_DISC,PROD_OFF_PRICE,TOTAL_TX_AMT,PROD_UNITS,PROD_WEIGHT,UNIT_PRICE_DSCNT_PCT,DSCNT_AMT,OL_SALES_PRICE,PROD_COLOR,ITM_ID,ITM_NAME,PRMTION_ID,PRMTION_NAME,SHP_MODE_ID,SHP_MODE,DELIVERY_COUNTRY,DELIVERY_STATE,DELIVERY_CITY,DELIVERY_DISTRICT,TRACKING_NO,ACTIVE_EMUI_VERSION,WH_NAME,STR_ORDER_NO,STR_ORDER_DATE,OL_ORDER_NO,OL_ORDER_DATE,OL_SITE,CUST_NICK_NAME,CUST_FIRST_NAME,CUST_LAST_NAME,CUST_PRFRD_FLG,CUST_BIRTH_DY,CUST_BIRTH_MM,CUST_BIRTH_YR,CUST_BIRTH_COUNTRY,CUST_LOGIN,CUST_EMAIL_ADDR,CUST_LAST_RVW_DATE,CUST_SEX,CUST_ADDRESS_ID,CUST_STREET_NO,CUST_STREET_NAME,CUST_AGE,CUST_SUITE_NO,CUST_ZIP,CUST_COUNTY,PRODUCT_ID,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,PROD_SHELL_COLOR,DEVICE_NAME,PROD_SHORT_DESC,PROD_LONG_DESC,PROD_THUMB,PROD_IMAGE,PROD_UPDATE_DATE,PROD_BAR_CODE,PROD_LIVE,PROD_LOC,PROD_RAM,PROD_ROM,PROD_CPU_CLOCK,PROD_SERIES,ITM_REC_START_DATE,ITM_REC_END_DATE,ITM_BRAND_ID,ITM_BRAND,ITM_CLASS_ID,ITM_CLASS,ITM_CATEGORY_ID,ITM_CATEGORY,ITM_MANUFACT_ID,ITM_MANUFACT,ITM_FORMULATION,ITM_COLOR,ITM_CONTAINER,ITM_MANAGER_ID,PRM_START_DATE,PRM_END_DATE,PRM_CHANNEL_DMAIL,PRM_CHANNEL_EMAIL,PRM_CHANNEL_CAT,PRM_CHANNEL_TV,PRM_CHANNEL_RADIO,PRM_CHANNEL_PRESS,PRM_CHANNEL_EVENT,PRM_CHANNEL_DEMO,PRM_CHANNEL_DETAILS,PRM_PURPOSE,PRM_DSCNT_ACTIVE,SHP_CODE,SHP_CARRIER,SHP_CONTRACT,CHECK_DATE,CHECK_YR,CHECK_MM,CHECK_DY,CHECK_HOUR,BOM,INSIDE_NAME,PACKING_DATE,PACKING_YR,PACKING_MM,PACKING_DY,PACKING_HOUR,DELIVERY_PROVINCE,PACKING_LIST_NO,ACTIVE_CHECK_TIME,ACTIVE_CHECK_YR,ACTIVE_CHECK_MM,ACTIVE_CHECK_DY,ACTIVE_CHECK_HOUR,ACTIVE_NETWORK,ACTIVE_FIRMWARE_VER,ACTIVE_OS_VERSION,LATEST_CHECK_TIME,LATEST_CHECK_YR,LATEST_CHECK_MM,LATEST_CHECK_DY,LATEST_CHECK_HOUR,LATEST_FIRMWARE_VER,LATEST_EMUI_VERSION,LATEST_OS_VERSION,LATEST_NETWORK,WH_ID,WH_STREET_NO,WH_STREET_NAME,WH_STREET_TYPE,WH_SUITE_NO,WH_ZIP,OL_SITE_DESC,OL_RET_ORDER_NO,OL_RET_DATE,CUST_DEP_COUNT,CUST_VEHICLE_COUNT,CUST_ADDRESS_CNT,CUST_CRNT_CDEMO_CNT,CUST_CRNT_HDEMO_CNT,CUST_CRNT_ADDR_DM,CUST_FIRST_SHIPTO_CNT,CUST_FIRST_SALES_CNT,CUST_GMT_OFFSET,CUST_DEMO_CNT,CUST_INCOME,PROD_UNLIMITED,EXTENDED_AMT,FREIGHT_CHRG,WAITING_PERIOD,DELIVERY_PERIOD,ITM_CRNT_PRICE,ITM_UNITS,ITM_WSLE_CST,ITM_SIZE,PRM_CST,PRM_RESPONSE_TARGET,PRM_ITM_DM,SHP_MODE_CNT,WH_GMT_OFFSET,WH_SQ_FT,STR_ORD_QTY,STR_WSLE_CST,STR_LIST_PRICE,STR_SALES_PRICE,STR_EXT_DSCNT_AMT,STR_EXT_SALES_PRICE,STR_EXT_WSLE_CST,STR_EXT_LIST_PRICE,STR_EXT_TX,STR_COUPON_AMT,STR_NET_PAID,STR_NET_PAID_INC_TX,STR_NET_PRFT,STR_SOLD_YR_CNT,STR_SOLD_MM_CNT,STR_SOLD_ITM_CNT,STR_TOTAL_CUST_CNT,STR_AREA_CNT,STR_DEMO_CNT,STR_OFFER_CNT,STR_PRM_CNT,STR_TICKET_CNT,STR_NET_PRFT_DM_A,STR_NET_PRFT_DM_B,STR_NET_PRFT_DM_C,STR_NET_PRFT_DM_D,STR_NET_PRFT_DM_E,STR_RET_STR_ID,STR_RET_REASON_CNT,STR_RET_TICKET_NO,STR_RTRN_QTY,STR_RTRN_AMT,STR_RTRN_TX,STR_RTRN_AMT_INC_TX,STR_RET_FEE,STR_RTRN_SHIP_CST,STR_RFNDD_CSH,STR_REVERSED_CHRG,STR_STR_CREDIT,STR_RET_NET_LOSS,STR_RTRNED_YR_CNT,STR_RTRN_MM_CNT,STR_RET_ITM_CNT,STR_RET_CUST_CNT,STR_RET_AREA_CNT,STR_RET_OFFER_CNT,STR_RET_PRM_CNT,STR_RET_NET_LOSS_DM_A,STR_RET_NET_LOSS_DM_B,STR_RET_NET_LOSS_DM_C,STR_RET_NET_LOSS_DM_D,OL_ORD_QTY,OL_WSLE_CST,OL_LIST_PRICE,OL_EXT_DSCNT_AMT,OL_EXT_SALES_PRICE,OL_EXT_WSLE_CST,OL_EXT_LIST_PRICE,OL_EXT_TX,OL_COUPON_AMT,OL_EXT_SHIP_CST,OL_NET_PAID,OL_NET_PAID_INC_TX,OL_NET_PAID_INC_SHIP,OL_NET_PAID_INC_SHIP_TX,OL_NET_PRFT,OL_SOLD_YR_CNT,OL_SOLD_MM_CNT,OL_SHIP_DATE_CNT,OL_ITM_CNT,OL_BILL_CUST_CNT,OL_BILL_AREA_CNT,OL_BILL_DEMO_CNT,OL_BILL_OFFER_CNT,OL_SHIP_CUST_CNT,OL_SHIP_AREA_CNT,OL_SHIP_DEMO_CNT,OL_SHIP_OFFER_CNT,OL_WEB_PAGE_CNT,OL_WEB_SITE_CNT,OL_SHIP_MODE_CNT,OL_WH_CNT,OL_PRM_CNT,OL_NET_PRFT_DM_A,OL_NET_PRFT_DM_B,OL_NET_PRFT_DM_C,OL_NET_PRFT_DM_D,OL_RET_RTRN_QTY,OL_RTRN_AMT,OL_RTRN_TX,OL_RTRN_AMT_INC_TX,OL_RET_FEE,OL_RTRN_SHIP_CST,OL_RFNDD_CSH,OL_REVERSED_CHRG,OL_ACCOUNT_CREDIT,OL_RTRNED_YR_CNT,OL_RTRNED_MM_CNT,OL_RTRITM_CNT,OL_RFNDD_CUST_CNT,OL_RFNDD_AREA_CNT,OL_RFNDD_DEMO_CNT,OL_RFNDD_OFFER_CNT,OL_RTRNING_CUST_CNT,OL_RTRNING_AREA_CNT,OL_RTRNING_DEMO_CNT,OL_RTRNING_OFFER_CNT,OL_RTRWEB_PAGE_CNT,OL_REASON_CNT,OL_NET_LOSS,OL_NET_LOSS_DM_A,OL_NET_LOSS_DM_B,OL_NET_LOSS_DM_C')""").collect
     checkAnswer(s"""select count(*) from DL_HistoricalData""",
       Seq(Row(10000)), "DataLoadingTestCase_History_Data_Load_001_001-002-TC-01_749")
@@ -992,7 +981,7 @@
 
   //Check for the incremental load data DML loading the data successfully with correct syntax
   test("Incremental_Data_Load_001_001-001-TC-01_840", Include) {
-     sql(s""" CREATE TABLE DL_IncrementalLoad (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
+     sql(s""" CREATE TABLE DL_IncrementalLoad (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
 
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/join1.csv' into table DL_IncrementalLoad options ('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/join1.csv' into table DL_IncrementalLoad options ('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
@@ -1029,7 +1018,7 @@
 
   //Check data load with Column dictionary for int column
   test("Details-Loading-StreamLoad-002-002", Include) {
-     sql(s"""CREATE TABLE DL_NoIntInclude (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2')""").collect
+     sql(s"""CREATE TABLE DL_NoIntInclude (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
 
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/join1.csv'  into table DL_NoIntInclude OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOJ,DOB,BIGINT_COLUMN2,BIGINT_COLUMN1,DECIMAL_COLUMN2,DECIMAL_COLUMN1,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     checkAnswer(s"""select count(*)from DL_NoIntInclude""",
@@ -1135,7 +1124,7 @@
 
   //Test Double data type max value support by Carbon
   test("FN_Carbon_DecimalDataType_TC_01_126", Include) {
-     sql(s"""CREATE TABLE DL_MaxDoubleValue (val double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='val')""").collect
+     sql(s"""CREATE TABLE DL_MaxDoubleValue (val double) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/DoubleVal.csv' INTO TABLE DL_MaxDoubleValue OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='val')""").collect
     checkAnswer(s"""select count(*) from DL_MaxDoubleValue""",
       Seq(Row(4)), "DataLoadingTestCase_FN_Carbon_DecimalDataType_TC_01_126")
@@ -1406,7 +1395,7 @@
 
   //Check for the incremental load data DML without "QUOTECHAR" specified loading the data successfully.
   test("Incremental_Data_Load_001_001-001-TC-11_840", Include) {
-     sql(s"""CREATE TABLE DL_without_QUOTECHAR (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
+     sql(s"""CREATE TABLE DL_without_QUOTECHAR (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'  """).collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/InsertData/2000_UniqData.csv' into table DL_without_QUOTECHAR OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='REDIRECT','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     checkAnswer(s"""select count(*) from DL_without_QUOTECHAR""",
       Seq(Row(2010)), "DataLoadingTestCase_Incremental_Data_Load_001_001-001-TC-11_840")
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/GlobalSortTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/GlobalSortTestCase.scala
index 84a8f09..4e865ba 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/GlobalSortTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/GlobalSortTestCase.scala
@@ -200,7 +200,7 @@
     sql(s"""drop table if exists t3""").collect
     sql(s"""CREATE TABLE t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')""").collect
     sql(
-    s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/batchsort/data.csv' into table t3 options('SINGLE_PASS'='TRUE', 'GLOBAL_SORT_PARTITIONS'='2','COLUMNDICT'='country:$resourcesPath/Data/columndict/country.csv')""".stripMargin).collect
+    s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/batchsort/data.csv' into table t3 options('GLOBAL_SORT_PARTITIONS'='2')""".stripMargin).collect
 
     sql(s"""select * from t3""").collect
     sql(s"""drop table if exists t3""").collect
@@ -211,7 +211,7 @@
   test("Carbon-Loading-Optimizations-Global-Sort-01-01-16", Include) {
     sql(s"""drop table if exists t3""").collect
     sql(s"""CREATE TABLE t3 (ID Int, country String, name String, phonetype String, serialname String, salary Int,floatField float) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')""").collect
-    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/batchsort/data.csv' into table t3 options('SINGLE_PASS'='TRUE', 'GLOBAL_SORT_PARTITIONS'='2','ALL_DICTIONARY_PATH'='$resourcesPath/Data/columndict/data.dictionary')""").collect
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/batchsort/data.csv' into table t3 options('GLOBAL_SORT_PARTITIONS'='2')""").collect
 
     sql(s"""select * from t3""").collect
     sql(s"""drop table if exists t3""").collect
@@ -223,7 +223,7 @@
     sql(s"""drop table if exists uniqdata20b""").collect
     sql(s"""drop table if exists uniqdata20c""").collect
     sql(s"""CREATE TABLE uniqdata20b (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'carbondata'""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/7000_UniqData.csv' into table uniqdata20b OPTIONS('DELIMITER'=',' , 'SINGLE_PASS'='false','QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/7000_UniqData.csv' into table uniqdata20b OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     sql(s"""CREATE TABLE uniqdata20c (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'carbondata'""").collect
     sql(s"""insert into uniqdata20c select * from uniqdata20b""").collect
 
@@ -402,7 +402,7 @@
   test("Carbon-Loading-Optimizations-Global-Sort-01-01-36", Include) {
     sql(s"""drop TABLE if exists uniqdata_c""").collect
     sql(s"""CREATE TABLE uniqdata_c (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/7000_UniqData.csv' into table uniqdata_c OPTIONS('GLOBAL_SORT_PARTITIONS'='2','DELIMITER'=',' , 'SINGLE_PASS'='false','QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
+    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/7000_UniqData.csv' into table uniqdata_c OPTIONS('GLOBAL_SORT_PARTITIONS'='2','DELIMITER'=',' , 'QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     sql(s"""delete from uniqdata_c where CUST_NAME='CUST_NAME_20000'""").collect
 
     sql(s"""select * from uniqdata_c""").collect
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/InvertedindexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/InvertedindexTestCase.scala
index d9d35fb..af414ef 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/InvertedindexTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/InvertedindexTestCase.scala
@@ -37,47 +37,6 @@
      sql(s"""drop table if exists uniqdata""").collect
   }
 
-
-  //To check no_inverted_index with dictionary_include
-  test("NoInvertedindex-TC002", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure
-  test("NoInvertedindex-TC003", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude
-  test("NoInvertedindex-TC004", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include
-  test("NoInvertedindex-TC005", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include
-  test("NoInvertedindex-TC007", Include) {
-     sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
   //To check no_inverted_index with dimension
   test("NoInvertedindex-TC008", Include) {
     sql(s"""drop table if exists uniqdata""").collect
@@ -86,52 +45,6 @@
      sql(s"""drop table if exists uniqdata""").collect
   }
 
-
-  //To check no_inverted_index with dictionary_include
-  test("NoInvertedindex-TC009", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude
-  test("NoInvertedindex-TC010", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include
-  test("NoInvertedindex-TC011", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include
-  test("NoInvertedindex-TC013", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure
-  test("NoInvertedindex-TC014", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
   //To check no_inverted_index with dimension and limit
   test("NoInvertedindex-TC015", Include) {
     sql(s"""drop table if exists uniqdata""").collect
@@ -256,625 +169,6 @@
      sql(s"""drop table if exists uniqdata""").collect
   }
 
-
-  //To check no_inverted_index with dictionary_include and limit
-  test("NoInvertedindex-TC026", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata limit 100""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC026")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and count()
-  test("NoInvertedindex-TC027", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(7)), "invertedindexTestCase_NoInvertedindex-TC027")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and sum()
-  test("NoInvertedindex-TC028", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(INTEGER_COLUMN1) from uniqdata""",
-      Seq(Row(28)), "invertedindexTestCase_NoInvertedindex-TC028")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and >= operator
-  test("NoInvertedindex-TC029", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID >= 9001""",
-      Seq(Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC029")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and !=
-  test("NoInvertedindex-TC030", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID != 9001""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC030")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and between
-  test("NoInvertedindex-TC031", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id between 9002 and 9030""",
-      Seq(Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC031")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and like
-  test("NoInvertedindex-TC032", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id Like '9%'""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC032")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and join
-  test("NoInvertedindex-TC033", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""drop table if exists uniqdata1""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    sql(s"""select uniqdata.CUST_ID from uniqdata cross join uniqdata1 where uniqdata.CUST_ID > 9002 and uniqdata1.CUST_ID > 9003""").collect
-
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""drop table if exists uniqdata1""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and having
-  test("NoInvertedindex-TC034", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id from uniqdata where cust_id > 9000 group by cust_id having cust_id = 9002""",
-      Seq(Row(9002)), "invertedindexTestCase_NoInvertedindex-TC034")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and sortby
-  test("NoInvertedindex-TC035", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id > 9004 sort by cust_name desc""",
-      Seq(Row(9006,"CUST_NAME_00006"),Row(9005,"CUST_NAME_00005")), "invertedindexTestCase_NoInvertedindex-TC035")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and groupby
-  test("NoInvertedindex-TC036", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(CUST_ID) from uniqdata group by cust_id""",
-      Seq(Row(9006),Row(9001),Row(9004),Row(9002),Row(9005),Row(9003),Row(9000)), "invertedindexTestCase_NoInvertedindex-TC036")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and limit
-  test("NoInvertedindex-TC037", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata limit 100""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC037")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and count()
-  test("NoInvertedindex-TC038", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(7)), "invertedindexTestCase_NoInvertedindex-TC038")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and sum()
-  test("NoInvertedindex-TC039", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(INTEGER_COLUMN1) from uniqdata""",
-      Seq(Row(28)), "invertedindexTestCase_NoInvertedindex-TC039")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and >= operator
-  test("NoInvertedindex-TC040", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID >= 9001""",
-      Seq(Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC040")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and !=
-  test("NoInvertedindex-TC041", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID != 9001""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC041")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and between
-  test("NoInvertedindex-TC042", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id between 9002 and 9030""",
-      Seq(Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC042")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and like
-  test("NoInvertedindex-TC043", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id Like '9%'""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC043")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and join
-  test("NoInvertedindex-TC044", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""drop table if exists uniqdata1""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    sql(s"""select uniqdata.CUST_ID from uniqdata cross join uniqdata1 where uniqdata.CUST_ID > 9002 and uniqdata1.CUST_ID > 9003""").collect
-
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""drop table if exists uniqdata1""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and having
-  test("NoInvertedindex-TC045", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id from uniqdata where cust_id > 9000 group by cust_id having cust_id = 9002""",
-      Seq(Row(9002)), "invertedindexTestCase_NoInvertedindex-TC045")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include and measure and sortby
-  test("NoInvertedindex-TC046", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id > 9004 sort by cust_name desc""",
-      Seq(Row(9006,"CUST_NAME_00006"),Row(9005,"CUST_NAME_00005")), "invertedindexTestCase_NoInvertedindex-TC046")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_include measure and groupby
-  test("NoInvertedindex-TC047", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(cust_id) from uniqdata group by cust_id""",
-      Seq(Row(9006),Row(9001),Row(9004),Row(9002),Row(9005),Row(9003),Row(9000)), "invertedindexTestCase_NoInvertedindex-TC047")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and limit
-  test("NoInvertedindex-TC048", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata limit 100""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC048")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and count()
-  test("NoInvertedindex-TC049", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(7)), "invertedindexTestCase_NoInvertedindex-TC049")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and sum()
-  test("NoInvertedindex-TC050", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(INTEGER_COLUMN1) from uniqdata""",
-      Seq(Row(28)), "invertedindexTestCase_NoInvertedindex-TC050")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and >= operator
-  test("NoInvertedindex-TC051", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID >= 9001""",
-      Seq(Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC051")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and !=
-  test("NoInvertedindex-TC052", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID != 9001""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC052")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and between
-  test("NoInvertedindex-TC053", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id between 9002 and 9030""",
-      Seq(Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC053")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and like
-  test("NoInvertedindex-TC054", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id Like '9%'""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC054")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and join
-  test("NoInvertedindex-TC055", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""drop table if exists uniqdata1""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    sql(s"""select uniqdata.CUST_ID from uniqdata cross join uniqdata1 where uniqdata.CUST_ID > 9002 and uniqdata1.CUST_ID > 9003""").collect
-
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""drop table if exists uniqdata1""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and having
-  test("NoInvertedindex-TC056", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id from uniqdata where cust_id > 9000 group by cust_id having cust_id = 9002""",
-      Seq(Row(9002)), "invertedindexTestCase_NoInvertedindex-TC056")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and sortby
-  test("NoInvertedindex-TC057", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id > 9004 sort by cust_name desc""",
-      Seq(Row(9006,"CUST_NAME_00006"),Row(9005,"CUST_NAME_00005")), "invertedindexTestCase_NoInvertedindex-TC057")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and groupby
-  test("NoInvertedindex-TC058", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(cust_id) from uniqdata group by cust_id""",
-      Seq(Row(9006),Row(9001),Row(9004),Row(9002),Row(9005),Row(9003),Row(9000)), "invertedindexTestCase_NoInvertedindex-TC058")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and limit
-  test("NoInvertedindex-TC059", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata limit 100""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC059")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and count()
-  test("NoInvertedindex-TC060", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(7)), "invertedindexTestCase_NoInvertedindex-TC060")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and sum()
-  test("NoInvertedindex-TC061", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(INTEGER_COLUMN1) from uniqdata""",
-      Seq(Row(28)), "invertedindexTestCase_NoInvertedindex-TC061")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and >= operator
-  test("NoInvertedindex-TC062", Include) {
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID >= 9001""",
-      Seq(Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC062")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and !=
-  test("NoInvertedindex-TC063", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID != 9001""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC063")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and between
-  test("NoInvertedindex-TC064", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id between 9002 and 9030""",
-      Seq(Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC064")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and like
-  test("NoInvertedindex-TC065", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id Like '9%'""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC065")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and join
-  test("NoInvertedindex-TC066", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""drop table if exists uniqdata1""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    sql(s"""select uniqdata.CUST_ID from uniqdata cross join uniqdata1 where uniqdata.CUST_ID > 9002 and uniqdata1.CUST_ID > 9003""").collect
-
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""drop table if exists uniqdata1""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and having
-  test("NoInvertedindex-TC067", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id from uniqdata where cust_id > 9000 group by cust_id having cust_id = 9002""",
-      Seq(Row(9002)), "invertedindexTestCase_NoInvertedindex-TC067")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and sortby
-  test("NoInvertedindex-TC068", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id > 9004 sort by cust_name desc""",
-      Seq(Row(9006,"CUST_NAME_00006"),Row(9005,"CUST_NAME_00005")), "invertedindexTestCase_NoInvertedindex-TC068")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index with dictionary_exclude and dictionary_include and groupby
-  test("NoInvertedindex-TC069", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(cust_id) from uniqdata group by cust_id""",
-      Seq(Row(9006),Row(9001),Row(9004),Row(9002),Row(9005),Row(9003),Row(9000)), "invertedindexTestCase_NoInvertedindex-TC069")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and limit
-  test("NoInvertedindex-TC081", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata limit 100""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC081")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and count()
-  test("NoInvertedindex-TC082", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(7)), "invertedindexTestCase_NoInvertedindex-TC082")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and sum()
-  test("NoInvertedindex-TC083", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(INTEGER_COLUMN1) from uniqdata""",
-      Seq(Row(28)), "invertedindexTestCase_NoInvertedindex-TC083")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and >= operator
-  test("NoInvertedindex-TC084", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID >= 9001""",
-      Seq(Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC084")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and !=
-  test("NoInvertedindex-TC085", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where CUST_ID != 9001""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC085")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and between
-  test("NoInvertedindex-TC086", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id between 9002 and 9030""",
-      Seq(Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC086")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and like
-  test("NoInvertedindex-TC087", Include) {
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id Like '9%'""",
-      Seq(Row(9000,"CUST_NAME_00000"),Row(9001,"CUST_NAME_00001"),Row(9002,"CUST_NAME_00002"),Row(9003,"CUST_NAME_00003"),Row(9004,"CUST_NAME_00004"),Row(9005,"CUST_NAME_00005"),Row(9006,"CUST_NAME_00006")), "invertedindexTestCase_NoInvertedindex-TC087")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and join
-  test("NoInvertedindex-TC088", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""drop table if exists uniqdata1""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""CREATE TABLE uniqdata1 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    sql(s"""select uniqdata.CUST_ID from uniqdata cross join uniqdata1 where uniqdata.CUST_ID > 9002 and uniqdata1.CUST_ID > 9003""").collect
-
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""drop table if exists uniqdata1""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and having
-  test("NoInvertedindex-TC089", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id from uniqdata where cust_id > 9000 group by cust_id having cust_id = 9002""",
-      Seq(Row(9002)), "invertedindexTestCase_NoInvertedindex-TC089")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and sortby
-  test("NoInvertedindex-TC090", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select cust_id,cust_name from uniqdata where cust_id > 9004 sort by cust_name desc""",
-      Seq(Row(9006,"CUST_NAME_00006"),Row(9005,"CUST_NAME_00005")), "invertedindexTestCase_NoInvertedindex-TC090")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
-  //To check no_inverted_index for timestamp with dictionary_exclude and dictionary_include  and groupby
-  test("NoInvertedindex-TC091", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','DICTIONARY_EXCLUDE'='ACTIVE_EMUI_VERSION','NO_INVERTED_INDEX'='DOB')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select sum(cust_id) from uniqdata group by cust_id""",
-      Seq(Row(9006),Row(9001),Row(9004),Row(9002),Row(9005),Row(9003),Row(9000)), "invertedindexTestCase_NoInvertedindex-TC091")
-     sql(s"""drop table if exists uniqdata""").collect
-  }
-
-
   //To check no_inverted_index with measure
   test("NoInvertedindex-TC092", Include) {
      sql(s"""drop table if exists uniqdata""").collect
@@ -887,7 +181,7 @@
   test("NoInvertedindex-TC097", Include) {
     sql(s"""drop table if exists uniqdata""").collect
     intercept[Exception] {
-      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('COLUMN_GROUPS'='(CUST_NAME,ACTIVE_EMUI_VERSION)','DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_NAME')""").collect
+      sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('COLUMN_GROUPS'='(CUST_NAME,ACTIVE_EMUI_VERSION)', 'NO_INVERTED_INDEX'='CUST_NAME')""").collect
       sql(s"""Alter table uniqdata drop columns(BIGINT_COLUMN1)""").collect
       sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
       sql(s"""select BIGINT_COLUMN1 from uniqdata""").collect
@@ -907,7 +201,7 @@
   //to check dictionary exclude with no_inverted_index
   test("NoInvertedindex-TC102", Include) {
      sql(s"""drop table if exists uniqdata""").collect
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='CUST_NAME','NO_INVERTED_INDEX'='CUST_NAME')""").collect
+    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('NO_INVERTED_INDEX'='CUST_NAME')""").collect
      sql(s"""drop table if exists uniqdata""").collect
   }
 
@@ -967,7 +261,7 @@
   //To validate No Inverted Index on Low Cardinality Column
   test("NoInvertedindex-TC107", Include) {
     sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Double_COLUMN1','NO_INVERTED_INDEX'='Double_COLUMN1')""").collect
+     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('NO_INVERTED_INDEX'='Double_COLUMN1')""").collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     sql(s"""select * from uniqdata""").collect
 
@@ -1030,7 +324,7 @@
   //To validate No Inverted Index on Low Cardinality Column
   test("NoInvertedindex-TC112", Include) {
     sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Double_COLUMN1','NO_INVERTED_INDEX'='Double_COLUMN1')""").collect
+     sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('NO_INVERTED_INDEX'='Double_COLUMN1')""").collect
    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/noinverted.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
     sql(s"""select * from uniqdata""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala
index 199358b..a1c6887 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LoadTableWithLocalDictionaryTestCase.scala
@@ -79,9 +79,7 @@
 
   test("test local dictionary generation for local dictioanry include") {
     sql("drop table if exists local2")
-    sql(
-      "CREATE TABLE local2(name string) STORED BY 'carbondata' tblproperties" +
-      "('dictionary_include'='name')")
+    sql("CREATE TABLE local2(name string) STORED BY 'carbondata' ")
     sql("load data inpath '" + file1 + "' into table local2 OPTIONS('header'='false')")
     assert(!checkForLocalDictionary(getDimRawChunk(0)))
   }
@@ -90,7 +88,7 @@
     sql("drop table if exists local2")
     sql(
       "CREATE TABLE local2(name string) STORED BY 'carbondata' tblproperties" +
-      "('local_dictionary_enable'='true','dictionary_exclude'='name')")
+      "('local_dictionary_enable'='true')")
     sql("load data inpath '" + file1 + "' into table local2 OPTIONS('header'='false')")
     assert(checkForLocalDictionary(getDimRawChunk(0)))
   }
@@ -299,8 +297,7 @@
       val decoder = encodingFactory.createDecoder(encodings, encoderMetas, compressorName)
       val dictionaryPage = decoder
         .decode(local_dictionary.getDictionary_data, 0, local_dictionary.getDictionary_data.length)
-      val dictionaryMap = new
-          util.HashMap[DictionaryByteArrayWrapper, Integer]
+      val dictionaryMap = new util.HashMap[DictionaryByteArrayWrapper, Integer]
       val usedDictionaryValues = util.BitSet
         .valueOf(CompressorFactory.getInstance.getCompressor(compressorName)
           .unCompressByte(local_dictionary.getDictionary_values))
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
index 54d0d11..8cf840d 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/LuceneTestCase.scala
@@ -56,9 +56,7 @@
     sql("DROP TABLE IF EXISTS datamap_main")
     sql(
       "CREATE TABLE datamap_main (id Int, date date, country string,name String, phonetype " +
-      "string, " +
-      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' " +
-      "tblproperties('dictionary_include'='country')")
+      "string, serialname String,salary int ) STORED BY 'org.apache.carbondata.format' ")
     val exception_otherdataType: Exception = intercept[Exception] {
       sql(
         s"""
@@ -164,7 +162,7 @@
          | DMProperties('INDEX_COLUMNS'='country')
       """.stripMargin)
     sql(s"LOAD DATA LOCAL INPATH '$csvPath' INTO TABLE datamap_main OPTIONS('header'='false'," +
-        s"'BAD_RECORDS_LOGGER_ENABLE'='FALSE','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE')")
+        s"'BAD_RECORDS_LOGGER_ENABLE'='FALSE','BAD_RECORDS_ACTION'='FORCE')")
     checkAnswer(sql("SELECT COUNT(*) FROM datamap_main WHERE TEXT_MATCH('country:china')"),
       sql("select COUNT(*) from datamap_main where country='china'"))
     sql("drop datamap if exists lucene_datamap on table datamap_main")
@@ -195,27 +193,6 @@
     sql("drop datamap if exists lucene_datamap on table datamap_main")
   }
 
-  //Check Lucene DataMap when Dictionary_Include is provided for TEXT_COLUMN in Main Table
-  test("LuceneDataMap_TC008", Include) {
-    sql("DROP TABLE IF EXISTS datamap_main")
-    sql(
-      "CREATE TABLE datamap_main (id Int, date string, country string,name String, phonetype " +
-      "string, " +
-      "serialname String,salary int ) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-      "('SORT_COLUMNS'='country,name','SORT_SCOPE'='LOCAL_SORT','DICTIONARY_INCLUDE'='country')")
-    val exception_dicitionaryinclude: Exception = intercept[Exception] {
-      sql(
-        s"""
-           | CREATE DATAMAP lucene_datamap ON TABLE datamap_main
-           | USING 'lucene'
-           | DMProperties('INDEX_COLUMNS'='country')
-      """.stripMargin)
-    }
-    assert(exception_dicitionaryinclude.getMessage
-      .contains("Dictionary column is not supported, column 'country' is dictionary column"))
-    sql("drop datamap if exists lucene_datamap on table datamap_main")
-  }
-
   override protected def afterAll(): Unit = {
     sql("DROP TABLE IF EXISTS datamap_main")
   }
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
index 5ab88bc..11a9483 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
@@ -53,14 +53,14 @@
       .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
     sql(s"""drop table if exists carbon_automation_nonmerge""").collect
 
-    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") >= 1)
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "true")
     sql("DROP TABLE IF EXISTS carbon_automation_merge")
-    sql(s"""create table carbon_automation_merge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+    sql(s"""create table carbon_automation_merge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_merge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
 
@@ -74,7 +74,7 @@
       .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT, "false")
     sql(s"""drop table if exists carbon_automation_nonmerge""").collect
 
-    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
@@ -93,7 +93,7 @@
 
     sql(s"""drop table if exists carbon_automation_nonmerge""").collect
 
-    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+    sql(s"""create table carbon_automation_nonmerge (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_nonmerge OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PreAggregateTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PreAggregateTestCase.scala
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PreAggregateTestCase.scala
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBVATestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBVATestCase.scala
index f0c5599..afa5a30 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBVATestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBVATestCase.scala
@@ -33,7 +33,7 @@
     sql(s"""drop table if exists Test_Boundary""").collect
     sql(s"""drop table if exists Test_Boundary_hive""").collect
 
-    sql(s"""create table Test_Boundary (c1_int int,c2_Bigint Bigint,c3_Decimal Decimal(38,30),c4_double double,c5_string string,c6_Timestamp Timestamp,c7_Datatype_Desc string) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='c6_Timestamp')""").collect
+    sql(s"""create table Test_Boundary (c1_int int,c2_Bigint Bigint,c3_Decimal Decimal(38,30),c4_double double,c5_string string,c6_Timestamp Timestamp,c7_Datatype_Desc string) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""create table Test_Boundary_hive (c1_int int,c2_Bigint Bigint,c3_Decimal Decimal(38,30),c4_double double,c5_string string,c6_Timestamp Timestamp,c7_Datatype_Desc string)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBasicTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBasicTestCase.scala
index ed58e70..47314bc 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBasicTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesBasicTestCase.scala
@@ -39,7 +39,7 @@
     sql(s"""drop table if exists uniqdata""").collect
     sql(s"""drop table if exists uniqdata_hive""").collect
 
-    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='DOB,DOJ')""").collect
+    sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""CREATE TABLE uniqdata_hive (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
@@ -5293,7 +5293,7 @@
     sql(s"""drop table if exists Carbon_automation""").collect
     sql(s"""drop table if exists Carbon_automation_hive""").collect
 
-    sql(s"""create table Carbon_automation (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double,imei_count int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+    sql(s"""create table Carbon_automation (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double,imei_count int) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""create table Carbon_automation_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, contractNumber double,ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointId double,gamePointDescription string,imei_count int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
   }
@@ -5312,7 +5312,7 @@
   test("TC_0000", Include) {
     sql(s"""drop table if exists Carbon_automation1""").collect
 
-    sql(s"""create table Carbon_automation1 (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId,Latest_YEAR,Latest_MONTH,Latest_DAY')""").collect
+    sql(s"""create table Carbon_automation1 (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' """).collect
 
   }
 
@@ -13080,7 +13080,7 @@
     sql(s"""drop table if exists pushupfilter""").collect
     sql(s"""drop table if exists pushupfilter_hive""").collect
 
-    sql(s"""create table pushupfilter (imei String,uuid String,MAC String,device_color String,device_shell_color String,device_name String,product_name String,ram String,rom String,cpu_clock String,series String,check_date String,check_year int,check_month int ,check_day int,check_hour int,bom String,inside_name String,packing_date String,packing_year String,packing_month String,packing_day String,packing_hour String,customer_name String,deliveryAreaId String,deliveryCountry String,deliveryProvince String,deliveryCity String,deliveryDistrict String,packing_list_no String,order_no String,Active_check_time String,Active_check_year int,Active_check_month int,Active_check_day int,Active_check_hour int,ActiveAreaId String,ActiveCountry String,ActiveProvince String,Activecity String,ActiveDistrict String,Active_network String,Active_firmware_version String,Active_emui_version String,Active_os_version String,Latest_check_time String,Latest_check_year int,Latest_check_month int,Latest_check_day int,Latest_check_hour int,Latest_areaId String,Latest_country String,Latest_province String,Latest_city String,Latest_district String,Latest_firmware_version String,Latest_emui_version String,Latest_os_version String,Latest_network String,site String,site_desc String,product String,product_desc String) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='check_year,check_month,check_day,check_hour,Active_check_year,Active_check_month,Active_check_day,Active_check_hour,Latest_check_year,Latest_check_month,Latest_check_day')""").collect
+    sql(s"""create table pushupfilter (imei String,uuid String,MAC String,device_color String,device_shell_color String,device_name String,product_name String,ram String,rom String,cpu_clock String,series String,check_date String,check_year int,check_month int ,check_day int,check_hour int,bom String,inside_name String,packing_date String,packing_year String,packing_month String,packing_day String,packing_hour String,customer_name String,deliveryAreaId String,deliveryCountry String,deliveryProvince String,deliveryCity String,deliveryDistrict String,packing_list_no String,order_no String,Active_check_time String,Active_check_year int,Active_check_month int,Active_check_day int,Active_check_hour int,ActiveAreaId String,ActiveCountry String,ActiveProvince String,Activecity String,ActiveDistrict String,Active_network String,Active_firmware_version String,Active_emui_version String,Active_os_version String,Latest_check_time String,Latest_check_year int,Latest_check_month int,Latest_check_day int,Latest_check_hour int,Latest_areaId String,Latest_country String,Latest_province String,Latest_city String,Latest_district String,Latest_firmware_version String,Latest_emui_version String,Latest_os_version String,Latest_network String,site String,site_desc String,product String,product_desc String) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""create table pushupfilter_hive (imei String,uuid String,MAC String,device_color String,device_shell_color String,device_name String,product_name String,ram String,rom String,cpu_clock String,series String,check_date String,check_year int,check_month int ,check_day int,check_hour int,bom String,inside_name String,packing_date String,packing_year String,packing_month String,packing_day String,packing_hour String,customer_name String,deliveryAreaId String,deliveryCountry String,deliveryProvince String,deliveryCity String,deliveryDistrict String,packing_list_no String,order_no String,Active_check_time String,Active_check_year int,Active_check_month int,Active_check_day int,Active_check_hour int,ActiveAreaId String,ActiveCountry String,ActiveProvince String,Activecity String,ActiveDistrict String,Active_network String,Active_firmware_version String,Active_emui_version String,Active_os_version String,Latest_check_time String,Latest_check_year int,Latest_check_month int,Latest_check_day int,Latest_check_hour int,Latest_areaId String,Latest_country String,Latest_province String,Latest_city String,Latest_district String,Latest_firmware_version String,Latest_emui_version String,Latest_os_version String,Latest_network String,site String,site_desc String,product String,product_desc String)ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesCompactionTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesCompactionTestCase.scala
index cb275d6..bfa1485 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesCompactionTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesCompactionTestCase.scala
@@ -33,7 +33,7 @@
     sql(s"""drop table if exists Comp_DICTIONARY_INCLUDE""").collect
     sql(s"""drop table if exists Comp_DICTIONARY_INCLUDE_hive""").collect
 
-    sql(s"""create table  Comp_DICTIONARY_INCLUDE (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='imei,deviceInformationId,productionDate,gamePointId,Latest_DAY,contractNumber')
+    sql(s"""create table  Comp_DICTIONARY_INCLUDE (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format'
   """).collect
 
     sql(s"""create table  Comp_DICTIONARY_INCLUDE_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string,deliveryTime string,channelsId string,channelsName string,deliveryAreaId string,deliveryCountry string,deliveryProvince string,deliveryCity string,deliveryDistrict string,deliveryStreet string,oxSingleNumber string,contractNumber BigInt,ActiveCheckTime string,ActiveAreaId string,ActiveCountry string,ActiveProvince string,Activecity string,ActiveDistrict string,ActiveStreet string,ActiveOperatorId string,Active_releaseId string,Active_EMUIVersion string,Active_operaSysVersion string,Active_BacVerNumber string,Active_BacFlashVer string,Active_webUIVersion string,Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string,Active_operatorsVersion string,Active_phonePADPartitionedVersions string,Latest_YEAR int,Latest_MONTH int,Latest_DAY Decimal(30,10),Latest_HOUR string,Latest_areaId string,Latest_country string,Latest_province string,Latest_city string,Latest_district string,Latest_street string,Latest_releaseId string,Latest_EMUIVersion string,Latest_operaSysVersion string,Latest_BacVerNumber string,Latest_BacFlashVer string,Latest_webUIVersion string,Latest_webUITypeCarrVer string,Latest_webTypeDataVerNumber string,Latest_operatorsVersion string,Latest_phonePADPartitionedVersions string,Latest_operatorId string,gamePointId double,gamePointDescription string) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
@@ -274,7 +274,7 @@
     sql(s"""drop table if exists Comp_DICTIONARY_EXCLUDE""").collect
     sql(s"""drop table if exists Comp_DICTIONARY_EXCLUDE_hive""").collect
 
-    sql(s"""create table  Comp_DICTIONARY_EXCLUDE (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='imei', 'DICTIONARY_INCLUDE'='productionDate')""").collect
+    sql(s"""create table  Comp_DICTIONARY_EXCLUDE (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""create table  Comp_DICTIONARY_EXCLUDE_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string,deliveryTime string,channelsId string,channelsName string,deliveryAreaId string,deliveryCountry string,deliveryProvince string,deliveryCity string,deliveryDistrict string,deliveryStreet string,oxSingleNumber string,contractNumber BigInt,ActiveCheckTime string,ActiveAreaId string,ActiveCountry string,ActiveProvince string,Activecity string,ActiveDistrict string,ActiveStreet string,ActiveOperatorId string,Active_releaseId string,Active_EMUIVersion string,Active_operaSysVersion string,Active_BacVerNumber string,Active_BacFlashVer string,Active_webUIVersion string,Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string,Active_operatorsVersion string,Active_phonePADPartitionedVersions string,Latest_YEAR int,Latest_MONTH int,Latest_DAY Decimal(30,10),Latest_HOUR string,Latest_areaId string,Latest_country string,Latest_province string,Latest_city string,Latest_district string,Latest_street string,Latest_releaseId string,Latest_EMUIVersion string,Latest_operaSysVersion string,Latest_BacVerNumber string,Latest_BacFlashVer string,Latest_webUIVersion string,Latest_webUITypeCarrVer string,Latest_webTypeDataVerNumber string,Latest_operatorsVersion string,Latest_phonePADPartitionedVersions string,Latest_operatorId string,gamePointId double,gamePointDescription string) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesExcludeDictionaryTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesExcludeDictionaryTestCase.scala
deleted file mode 100644
index 49d5b3e..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesExcludeDictionaryTestCase.scala
+++ /dev/null
@@ -1,5044 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import org.apache.spark.sql.common.util._
-import org.scalatest.BeforeAndAfterAll
-
-/**
- * Test Class for QueriesExcludeDictionaryTestCase to verify all scenerios
- */
-
-class QueriesExcludeDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
-
-
-  //DICTIONARY_EXCLUDE_CreateCube
-  test("Queries_DICTIONARY_EXCLUDE_CreateCube", Include) {
-    sql(s"""drop table if exists TABLE_DICTIONARY_EXCLUDE""").collect
-    sql(s"""drop table if exists TABLE_DICTIONARY_EXCLUDE1_hive""").collect
-
-    sql(s"""create table  TABLE_DICTIONARY_EXCLUDE (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='imei', 'DICTIONARY_INCLUDE'='productionDate')""").collect
-
-    sql(s"""create table  TABLE_DICTIONARY_EXCLUDE1_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string,deliveryTime string,channelsId string,channelsName string,deliveryAreaId string,deliveryCountry string,deliveryProvince string,deliveryCity string,deliveryDistrict string,deliveryStreet string,oxSingleNumber string,contractNumber BigInt,ActiveCheckTime string,ActiveAreaId string,ActiveCountry string,ActiveProvince string,Activecity string,ActiveDistrict string,ActiveStreet string,ActiveOperatorId string,Active_releaseId string,Active_EMUIVersion string,Active_operaSysVersion string,Active_BacVerNumber string,Active_BacFlashVer string,Active_webUIVersion string,Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string,Active_operatorsVersion string,Active_phonePADPartitionedVersions string,Latest_YEAR int,Latest_MONTH int,Latest_DAY Decimal(30,10),Latest_HOUR string,Latest_areaId string,Latest_country string,Latest_province string,Latest_city string,Latest_district string,Latest_street string,Latest_releaseId string,Latest_EMUIVersion string,Latest_operaSysVersion string,Latest_BacVerNumber string,Latest_BacFlashVer string,Latest_webUIVersion string,Latest_webUITypeCarrVer string,Latest_webTypeDataVerNumber string,Latest_operatorsVersion string,Latest_phonePADPartitionedVersions string,Latest_operatorId string,gamePointId double,gamePointDescription string) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
-
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_CreateCube_count
-  test("Queries_DICTIONARY_EXCLUDE_CreateCube_count", Include) {
-
-    sql(s"""select count(*) from TABLE_DICTIONARY_EXCLUDE""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_DataLoad
-  test("Queries_DICTIONARY_EXCLUDE_DataLoad", Include) {
-
-    sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/100_olap_C20.csv' INTO table TABLE_DICTIONARY_EXCLUDE options ('DELIMITER'=',', 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
-
-    sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/100_olap_C20_hive41.csv' INTO table TABLE_DICTIONARY_EXCLUDE1_hive """).collect
-
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_CreateCube1
-  test("Queries_DICTIONARY_EXCLUDE_CreateCube1", Include) {
-    sql(s"""drop table if exists TABLE_DICTIONARY_EXCLUDE1""").collect
-
-    sql(s"""create table  TABLE_DICTIONARY_EXCLUDE1 (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='imei')""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_DataLoad1
-  test("Queries_DICTIONARY_EXCLUDE_DataLoad1", Include) {
-
-    sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/100_olap_C20.csv' INTO table TABLE_DICTIONARY_EXCLUDE1 options ('DELIMITER'=',', 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_001
-  test("Queries_DICTIONARY_EXCLUDE_001", Include) {
-
-    checkAnswer(s"""Select count(imei) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""Select count(imei) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_001")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_002
-  test("Queries_DICTIONARY_EXCLUDE_002", Include) {
-
-    checkAnswer(s"""select count(DISTINCT imei) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT imei) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_002")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_003
-  test("Queries_DICTIONARY_EXCLUDE_003", Include) {
-
-    checkAnswer(s"""select sum(Latest_month)+10 as a ,imei  from TABLE_DICTIONARY_EXCLUDE group by imei order by imei""",
-      s"""select sum(Latest_month)+10 as a ,imei  from TABLE_DICTIONARY_EXCLUDE1_hive group by imei order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_003")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_004
-  test("Queries_DICTIONARY_EXCLUDE_004", Include) {
-
-    checkAnswer(s"""select max(imei),min(imei) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(imei),min(imei) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_004")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_005
-  test("Queries_DICTIONARY_EXCLUDE_005", Include) {
-
-    checkAnswer(s"""select min(imei), max(imei) Total from TABLE_DICTIONARY_EXCLUDE group by  channelsId order by Total""",
-      s"""select min(imei), max(imei) Total from TABLE_DICTIONARY_EXCLUDE1_hive group by  channelsId order by Total""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_005")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_006
-  test("Queries_DICTIONARY_EXCLUDE_006", Include) {
-
-    sql(s"""select last(imei) a from TABLE_DICTIONARY_EXCLUDE  group by imei order by imei limit 1""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_007
-  test("Queries_DICTIONARY_EXCLUDE_007", Include) {
-
-    sql(s"""select FIRST(imei) a from TABLE_DICTIONARY_EXCLUDE group by imei order by imei limit 1""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_008
-  test("Queries_DICTIONARY_EXCLUDE_008", Include) {
-
-    checkAnswer(s"""select imei,count(imei) a from TABLE_DICTIONARY_EXCLUDE group by imei order by imei""",
-      s"""select imei,count(imei) a from TABLE_DICTIONARY_EXCLUDE1_hive group by imei order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_008")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_009
-  test("Queries_DICTIONARY_EXCLUDE_009", Include) {
-
-    checkAnswer(s"""select Lower(imei) a  from TABLE_DICTIONARY_EXCLUDE order by imei""",
-      s"""select Lower(imei) a  from TABLE_DICTIONARY_EXCLUDE1_hive order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_009")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_010
-  test("Queries_DICTIONARY_EXCLUDE_010", Include) {
-
-    checkAnswer(s"""select distinct imei from TABLE_DICTIONARY_EXCLUDE order by imei""",
-      s"""select distinct imei from TABLE_DICTIONARY_EXCLUDE1_hive order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_010")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_011
-  test("Queries_DICTIONARY_EXCLUDE_011", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE order by imei limit 101 """,
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive order by imei limit 101 """, "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_011")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_012
-  test("Queries_DICTIONARY_EXCLUDE_012", Include) {
-
-    sql(s"""select imei as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_013
-  test("Queries_DICTIONARY_EXCLUDE_013", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where  (contractNumber == 9223372047700) and (imei=='1AA100004')""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where  (contractNumber == 9223372047700) and (imei=='1AA100004')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_013")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_014
-  test("Queries_DICTIONARY_EXCLUDE_014", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei !='1AA100064' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei !='1AA100064' order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_014")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_015
-  test("Queries_DICTIONARY_EXCLUDE_015", Include) {
-
-    checkAnswer(s"""select imei  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select imei  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_015")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_016
-  test("Queries_DICTIONARY_EXCLUDE_016", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei !='1AA100012' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei !='1AA100012' order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_016")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_017
-  test("Queries_DICTIONARY_EXCLUDE_017", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei >'1AA100012' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei >'1AA100012' order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_017")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_018
-  test("Queries_DICTIONARY_EXCLUDE_018", Include) {
-
-    checkAnswer(s"""select imei  from TABLE_DICTIONARY_EXCLUDE where imei<>imei""",
-      s"""select imei  from TABLE_DICTIONARY_EXCLUDE1_hive where imei<>imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_018")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_019
-  test("Queries_DICTIONARY_EXCLUDE_019", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei != Latest_areaId order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei != Latest_areaId order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_019")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_020
-  test("Queries_DICTIONARY_EXCLUDE_020", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where Latest_areaId<imei order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_areaId<imei order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_020")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_021
-  test("Queries_DICTIONARY_EXCLUDE_021", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where Latest_DAY<=imei order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY<=imei order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_021")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_022
-  test("Queries_DICTIONARY_EXCLUDE_022", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei <'1AA10002' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei <'1AA10002' order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_022")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_023
-  test("Queries_DICTIONARY_EXCLUDE_023", Include) {
-
-    checkAnswer(s"""select Latest_day  from TABLE_DICTIONARY_EXCLUDE where imei IS NULL""",
-      s"""select Latest_day  from TABLE_DICTIONARY_EXCLUDE1_hive where imei IS NULL""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_023")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_024
-  test("Queries_DICTIONARY_EXCLUDE_024", Include) {
-
-    checkAnswer(s"""select Latest_day  from TABLE_DICTIONARY_EXCLUDE where imei IS NOT NULL order by Latest_day""",
-      s"""select Latest_day  from TABLE_DICTIONARY_EXCLUDE1_hive where imei IS NOT NULL order by Latest_day""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_024")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_025
-  test("Queries_DICTIONARY_EXCLUDE_025", Include) {
-
-    checkAnswer(s"""Select count(imei),min(imei) from TABLE_DICTIONARY_EXCLUDE """,
-      s"""Select count(imei),min(imei) from TABLE_DICTIONARY_EXCLUDE1_hive """, "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_025")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_026
-  test("Queries_DICTIONARY_EXCLUDE_026", Include) {
-
-    checkAnswer(s"""select count(DISTINCT imei,latest_day) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT imei,latest_day) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_026")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_027
-  test("Queries_DICTIONARY_EXCLUDE_027", Include) {
-
-    checkAnswer(s"""select max(imei),min(imei),count(imei) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(imei),min(imei),count(imei) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_027")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_028
-  test("Queries_DICTIONARY_EXCLUDE_028", Include) {
-
-    checkAnswer(s"""select sum(imei),avg(imei),count(imei) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(imei),avg(imei),count(imei) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_028")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_029
-  test("Queries_DICTIONARY_EXCLUDE_029", Include) {
-
-    sql(s"""select last(imei),Min(imei),max(imei)  a from TABLE_DICTIONARY_EXCLUDE  order by a""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_030
-  test("Queries_DICTIONARY_EXCLUDE_030", Include) {
-
-    sql(s"""select FIRST(imei),Last(imei) a from TABLE_DICTIONARY_EXCLUDE group by imei order by imei limit 1""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_031
-  test("Queries_DICTIONARY_EXCLUDE_031", Include) {
-
-    checkAnswer(s"""select imei,count(imei) a from TABLE_DICTIONARY_EXCLUDE group by imei order by imei""",
-      s"""select imei,count(imei) a from TABLE_DICTIONARY_EXCLUDE1_hive group by imei order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_031")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_032
-  test("Queries_DICTIONARY_EXCLUDE_032", Include) {
-
-    checkAnswer(s"""select Lower(imei),upper(imei)  a  from TABLE_DICTIONARY_EXCLUDE order by imei""",
-      s"""select Lower(imei),upper(imei)  a  from TABLE_DICTIONARY_EXCLUDE1_hive order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_032")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_033
-  test("Queries_DICTIONARY_EXCLUDE_033", Include) {
-
-    checkAnswer(s"""select imei as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""",
-      s"""select imei as a from TABLE_DICTIONARY_EXCLUDE1_hive  order by a asc limit 10""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_033")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_034
-  test("Queries_DICTIONARY_EXCLUDE_034", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where  (contractNumber == 9223372047700) and (imei=='1AA100012')""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where  (contractNumber == 9223372047700) and (imei=='1AA100012')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_034")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_035
-  test("Queries_DICTIONARY_EXCLUDE_035", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei !='8imei' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei !='8imei' order by imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_035")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_036
-  test("Queries_DICTIONARY_EXCLUDE_036", Include) {
-
-    checkAnswer(s"""select imei  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select imei  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_036")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_037
-  test("Queries_DICTIONARY_EXCLUDE_037", Include) {
-
-    checkAnswer(s"""Select count(contractNumber) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""Select count(contractNumber) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_037")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_038
-  test("Queries_DICTIONARY_EXCLUDE_038", Include) {
-
-    checkAnswer(s"""select count(DISTINCT contractNumber) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT contractNumber) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_038")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_039
-  test("Queries_DICTIONARY_EXCLUDE_039", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)+10 as a ,contractNumber  from TABLE_DICTIONARY_EXCLUDE group by contractNumber""",
-      s"""select sum(contractNumber)+10 as a ,contractNumber  from TABLE_DICTIONARY_EXCLUDE1_hive group by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_039")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_040
-  test("Queries_DICTIONARY_EXCLUDE_040", Include) {
-
-    checkAnswer(s"""select max(contractNumber),min(contractNumber) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(contractNumber),min(contractNumber) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_040")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_041
-  test("Queries_DICTIONARY_EXCLUDE_041", Include) {
-
-    checkAnswer(s"""select sum(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_041")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_042
-  test("Queries_DICTIONARY_EXCLUDE_042", Include) {
-
-    checkAnswer(s"""select avg(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select avg(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_042")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_043
-  test("Queries_DICTIONARY_EXCLUDE_043", Include) {
-
-    checkAnswer(s"""select min(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select min(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_043")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_044
-  test("Queries_DICTIONARY_EXCLUDE_044", Include) {
-
-    sql(s"""select variance(contractNumber) as a   from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_045
-  ignore("Queries_DICTIONARY_EXCLUDE_045", Include) {
-
-    checkAnswer(s"""select var_pop(contractNumber) as a from (select * from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""",
-      s"""select var_pop(contractNumber) as a from (select * from TABLE_DICTIONARY_EXCLUDE1_hive order by contractNumber) t""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_045")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_046
-  ignore("Queries_DICTIONARY_EXCLUDE_046", Include) {
-
-    checkAnswer(s"""select var_samp(contractNumber) as a from (select * from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""",
-      s"""select var_samp(contractNumber) as a from (select * from TABLE_DICTIONARY_EXCLUDE1_hive order by contractNumber) t""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_046")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_047
-  test("Queries_DICTIONARY_EXCLUDE_047", Include) {
-
-    sql(s"""select stddev_pop(contractNumber) as a  from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_048
-  test("Queries_DICTIONARY_EXCLUDE_048", Include) {
-
-    sql(s"""select stddev_samp(contractNumber)  as a from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_049
-  test("Queries_DICTIONARY_EXCLUDE_049", Include) {
-
-    sql(s"""select covar_pop(contractNumber,contractNumber) as a  from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_050
-  test("Queries_DICTIONARY_EXCLUDE_050", Include) {
-
-    sql(s"""select covar_samp(contractNumber,contractNumber) as a  from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_051
-  test("Queries_DICTIONARY_EXCLUDE_051", Include) {
-
-    checkAnswer(s"""select corr(contractNumber,contractNumber)  as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select corr(contractNumber,contractNumber)  as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_051")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_052
-  test("Queries_DICTIONARY_EXCLUDE_052", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,0.2) as a  from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_053
-  test("Queries_DICTIONARY_EXCLUDE_053", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,0.2,5) as a  from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_054
-  test("Queries_DICTIONARY_EXCLUDE_054", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,array(0.2,0.3,0.99))  as a from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_055
-  test("Queries_DICTIONARY_EXCLUDE_055", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,array(0.2,0.3,0.99),5) as a from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_056
-  test("Queries_DICTIONARY_EXCLUDE_056", Include) {
-
-    sql(s"""select histogram_numeric(contractNumber,2)  as a from (select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_057
-  test("Queries_DICTIONARY_EXCLUDE_057", Include) {
-
-    checkAnswer(s"""select contractNumber+ 10 as a  from TABLE_DICTIONARY_EXCLUDE order by a""",
-      s"""select contractNumber+ 10 as a  from TABLE_DICTIONARY_EXCLUDE1_hive order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_057")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_058
-  test("Queries_DICTIONARY_EXCLUDE_058", Include) {
-
-    checkAnswer(s"""select min(contractNumber), max(contractNumber+ 10) Total from TABLE_DICTIONARY_EXCLUDE group by  channelsId order by Total""",
-      s"""select min(contractNumber), max(contractNumber+ 10) Total from TABLE_DICTIONARY_EXCLUDE1_hive group by  channelsId order by Total""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_058")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_059
-  test("Queries_DICTIONARY_EXCLUDE_059", Include) {
-
-    sql(s"""select last(contractNumber) a from TABLE_DICTIONARY_EXCLUDE  order by a""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_060
-  test("Queries_DICTIONARY_EXCLUDE_060", Include) {
-
-    sql(s"""select FIRST(contractNumber) a from TABLE_DICTIONARY_EXCLUDE order by a""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_061
-  test("Queries_DICTIONARY_EXCLUDE_061", Include) {
-
-    checkAnswer(s"""select contractNumber,count(contractNumber) a from TABLE_DICTIONARY_EXCLUDE group by contractNumber order by contractNumber""",
-      s"""select contractNumber,count(contractNumber) a from TABLE_DICTIONARY_EXCLUDE1_hive group by contractNumber order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_061")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_062
-  test("Queries_DICTIONARY_EXCLUDE_062", Include) {
-
-    checkAnswer(s"""select Lower(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE order by contractNumber""",
-      s"""select Lower(contractNumber) a  from TABLE_DICTIONARY_EXCLUDE1_hive order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_062")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_063
-  test("Queries_DICTIONARY_EXCLUDE_063", Include) {
-
-    checkAnswer(s"""select distinct contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber""",
-      s"""select distinct contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_063")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_064
-  test("Queries_DICTIONARY_EXCLUDE_064", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE order by contractNumber limit 101""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive order by contractNumber limit 101""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_064")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_065
-  test("Queries_DICTIONARY_EXCLUDE_065", Include) {
-
-    checkAnswer(s"""select contractNumber as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""",
-      s"""select contractNumber as a from TABLE_DICTIONARY_EXCLUDE1_hive  order by a asc limit 10""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_065")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_066
-  test("Queries_DICTIONARY_EXCLUDE_066", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where  (contractNumber == 9223372047700) and (imei=='1AA100012')""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where  (contractNumber == 9223372047700) and (imei=='1AA100012')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_066")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_067
-  test("Queries_DICTIONARY_EXCLUDE_067", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber !=9223372047700 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber !=9223372047700 order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_067")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_068
-  test("Queries_DICTIONARY_EXCLUDE_068", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color') order by contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color') order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_068")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_069
-  test("Queries_DICTIONARY_EXCLUDE_069", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber !=9223372047700 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber !=9223372047700 order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_069")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_070
-  test("Queries_DICTIONARY_EXCLUDE_070", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber >9223372047700 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber >9223372047700 order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_070")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_071
-  test("Queries_DICTIONARY_EXCLUDE_071", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE where contractNumber<>contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber<>contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_071")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_072
-  test("Queries_DICTIONARY_EXCLUDE_072", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber != Latest_areaId order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber != Latest_areaId order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_072")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_073
-  test("Queries_DICTIONARY_EXCLUDE_073", Include) {
-
-    checkAnswer(s"""select contractNumber, contractNumber from TABLE_DICTIONARY_EXCLUDE where Latest_areaId<contractNumber order by contractNumber""",
-      s"""select contractNumber, contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_areaId<contractNumber order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_073")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_074
-  test("Queries_DICTIONARY_EXCLUDE_074", Include) {
-
-    checkAnswer(s"""select contractNumber, contractNumber from TABLE_DICTIONARY_EXCLUDE where Latest_DAY<=contractNumber order by contractNumber""",
-      s"""select contractNumber, contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY<=contractNumber order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_074")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_075
-  test("Queries_DICTIONARY_EXCLUDE_075", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber <1000 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber <1000 order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_075")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_076
-  test("Queries_DICTIONARY_EXCLUDE_076", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber >1000 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber >1000 order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_076")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_077
-  test("Queries_DICTIONARY_EXCLUDE_077", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE where contractNumber IS NULL order by contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber IS NULL order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_077")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_078
-  test("Queries_DICTIONARY_EXCLUDE_078", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE where Latest_DAY IS NOT NULL order by contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY IS NOT NULL order by contractNumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_078")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_079
-  test("Queries_DICTIONARY_EXCLUDE_079", Include) {
-
-    checkAnswer(s"""Select count(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""Select count(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_079")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_080
-  test("Queries_DICTIONARY_EXCLUDE_080", Include) {
-
-    checkAnswer(s"""select count(DISTINCT Latest_DAY) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT Latest_DAY) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_080")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_081
-  test("Queries_DICTIONARY_EXCLUDE_081", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)+10 as a ,Latest_DAY  from TABLE_DICTIONARY_EXCLUDE group by Latest_DAY order by a""",
-      s"""select sum(Latest_DAY)+10 as a ,Latest_DAY  from TABLE_DICTIONARY_EXCLUDE1_hive group by Latest_DAY order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_081")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_082
-  test("Queries_DICTIONARY_EXCLUDE_082", Include) {
-
-    checkAnswer(s"""select max(Latest_DAY),min(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(Latest_DAY),min(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_082")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_083
-  test("Queries_DICTIONARY_EXCLUDE_083", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_083")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_084
-  test("Queries_DICTIONARY_EXCLUDE_084", Include) {
-
-    checkAnswer(s"""select avg(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select avg(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_084")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_085
-  test("Queries_DICTIONARY_EXCLUDE_085", Include) {
-
-    checkAnswer(s"""select min(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select min(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_085")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_086
-  test("Queries_DICTIONARY_EXCLUDE_086", Include) {
-
-    sql(s"""select variance(Latest_DAY) as a   from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_087
-  test("Queries_DICTIONARY_EXCLUDE_087", Include) {
-
-    sql(s"""select var_pop(Latest_DAY)  as a from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_088
-  test("Queries_DICTIONARY_EXCLUDE_088", Include) {
-
-    sql(s"""select var_samp(Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_089
-  test("Queries_DICTIONARY_EXCLUDE_089", Include) {
-
-    sql(s"""select stddev_pop(Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_090
-  test("Queries_DICTIONARY_EXCLUDE_090", Include) {
-
-    sql(s"""select stddev_samp(Latest_DAY)  as a from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_091
-  test("Queries_DICTIONARY_EXCLUDE_091", Include) {
-
-    sql(s"""select covar_pop(Latest_DAY,Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_092
-  test("Queries_DICTIONARY_EXCLUDE_092", Include) {
-
-    sql(s"""select covar_samp(Latest_DAY,Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_093
-  test("Queries_DICTIONARY_EXCLUDE_093", Include) {
-
-    checkAnswer(s"""select corr(Latest_DAY,Latest_DAY)  as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select corr(Latest_DAY,Latest_DAY)  as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_093")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_094
-  test("Queries_DICTIONARY_EXCLUDE_094", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,0.2) as a  from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_095
-  test("Queries_DICTIONARY_EXCLUDE_095", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,0.2,5) as a  from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_096
-  test("Queries_DICTIONARY_EXCLUDE_096", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,array(0.2,0.3,0.99))  as a from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_097
-  test("Queries_DICTIONARY_EXCLUDE_097", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,array(0.2,0.3,0.99),5) as a from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_098
-  test("Queries_DICTIONARY_EXCLUDE_098", Include) {
-
-    sql(s"""select histogram_numeric(Latest_DAY,2)  as a from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_099
-  test("Queries_DICTIONARY_EXCLUDE_099", Include) {
-
-    checkAnswer(s"""select Latest_DAY, Latest_DAY+ 10 as a  from TABLE_DICTIONARY_EXCLUDE order by a""",
-      s"""select Latest_DAY, Latest_DAY+ 10 as a  from TABLE_DICTIONARY_EXCLUDE1_hive order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_099")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_100
-  test("Queries_DICTIONARY_EXCLUDE_100", Include) {
-
-    checkAnswer(s"""select min(Latest_DAY) d, max(Latest_DAY+ 10) Total from TABLE_DICTIONARY_EXCLUDE group by  channelsId order by d, Total""",
-      s"""select min(Latest_DAY) d, max(Latest_DAY+ 10) Total from TABLE_DICTIONARY_EXCLUDE1_hive group by  channelsId order by d,Total""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_100")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_101
-  test("Queries_DICTIONARY_EXCLUDE_101", Include) {
-
-    sql(s"""select last(Latest_DAY) a from TABLE_DICTIONARY_EXCLUDE order by a""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_102
-  test("Queries_DICTIONARY_EXCLUDE_102", Include) {
-
-    sql(s"""select FIRST(Latest_DAY) a from (select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_103
-  test("Queries_DICTIONARY_EXCLUDE_103", Include) {
-
-    checkAnswer(s"""select Latest_DAY,count(Latest_DAY) a from TABLE_DICTIONARY_EXCLUDE group by Latest_DAY order by Latest_DAY""",
-      s"""select Latest_DAY,count(Latest_DAY) a from TABLE_DICTIONARY_EXCLUDE1_hive group by Latest_DAY order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_103")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_104
-  test("Queries_DICTIONARY_EXCLUDE_104", Include) {
-
-    checkAnswer(s"""select Lower(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE order by a""",
-      s"""select Lower(Latest_DAY) a  from TABLE_DICTIONARY_EXCLUDE1_hive order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_104")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_105
-  test("Queries_DICTIONARY_EXCLUDE_105", Include) {
-
-    checkAnswer(s"""select distinct Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY""",
-      s"""select distinct Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_105")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_106
-  test("Queries_DICTIONARY_EXCLUDE_106", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE order by Latest_DAY limit 101""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive order by Latest_DAY limit 101""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_106")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_107
-  test("Queries_DICTIONARY_EXCLUDE_107", Include) {
-
-    checkAnswer(s"""select Latest_DAY as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""",
-      s"""select Latest_DAY as a from TABLE_DICTIONARY_EXCLUDE1_hive  order by a asc limit 10""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_107")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_108
-  test("Queries_DICTIONARY_EXCLUDE_108", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where  (Latest_DAY == 1234567890123450.0000000000)  and (imei=='1AA1')""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where  (Latest_DAY == 1234567890123450.0000000000)  and (imei=='1AA1')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_108")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_109
-  test("Queries_DICTIONARY_EXCLUDE_109", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_109")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_110
-  test("Queries_DICTIONARY_EXCLUDE_110", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_110")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_111
-  test("Queries_DICTIONARY_EXCLUDE_111", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_111")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_112
-  test("Queries_DICTIONARY_EXCLUDE_112", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY >1234567890123450.0000000000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY >1234567890123450.0000000000  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_112")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_113
-  test("Queries_DICTIONARY_EXCLUDE_113", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE where Latest_DAY<>Latest_DAY""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY<>Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_113")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_114
-  test("Queries_DICTIONARY_EXCLUDE_114", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY != Latest_areaId order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY != Latest_areaId order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_114")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_115
-  test("Queries_DICTIONARY_EXCLUDE_115", Include) {
-
-    checkAnswer(s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_areaId<Latest_DAY order by Latest_DAY""",
-      s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_areaId<Latest_DAY order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_115")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_116
-  test("Queries_DICTIONARY_EXCLUDE_116", Include) {
-
-    checkAnswer(s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY<=Latest_DAY  order by Latest_DAY""",
-      s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY<=Latest_DAY  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_116")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_117
-  test("Queries_DICTIONARY_EXCLUDE_117", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY <1000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY <1000  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_117")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_118
-  test("Queries_DICTIONARY_EXCLUDE_118", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY >1000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY >1000  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_118")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_119
-  test("Queries_DICTIONARY_EXCLUDE_119", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE where Latest_DAY IS NULL  order by Latest_DAY""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY IS NULL  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_119")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_120
-  test("Queries_DICTIONARY_EXCLUDE_120", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE where Latest_DAY IS NOT NULL  order by Latest_DAY""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY IS NOT NULL  order by Latest_DAY""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_120")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_121
-  test("Queries_DICTIONARY_EXCLUDE_121", Include) {
-
-    checkAnswer(s"""Select count(gamePointId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""Select count(gamePointId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_121")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_122
-  test("Queries_DICTIONARY_EXCLUDE_122", Include) {
-
-    checkAnswer(s"""select count(DISTINCT gamePointId) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT gamePointId) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_122")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_123
-  test("Queries_DICTIONARY_EXCLUDE_123", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)+10 as a ,gamePointId  from TABLE_DICTIONARY_EXCLUDE group by gamePointId order by a""",
-      s"""select sum(gamePointId)+10 as a ,gamePointId  from TABLE_DICTIONARY_EXCLUDE1_hive group by gamePointId order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_123")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_124
-  test("Queries_DICTIONARY_EXCLUDE_124", Include) {
-
-    checkAnswer(s"""select max(gamePointId),min(gamePointId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(gamePointId),min(gamePointId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_124")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_125
-  ignore("Queries_DICTIONARY_EXCLUDE_125", Include) {
-
-    checkAnswer(s"""select sum(gamePointId) a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select sum(gamePointId) a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_125")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_126
-  ignore("Queries_DICTIONARY_EXCLUDE_126", Include) {
-
-    checkAnswer(s"""select avg(gamePointId) a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select avg(gamePointId) a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_126")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_127
-  test("Queries_DICTIONARY_EXCLUDE_127", Include) {
-
-    checkAnswer(s"""select min(gamePointId) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select min(gamePointId) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_127")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_128
-  test("Queries_DICTIONARY_EXCLUDE_128", Include) {
-
-    sql(s"""select variance(gamePointId) as a   from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_129
-  test("Queries_DICTIONARY_EXCLUDE_129", Include) {
-
-    sql(s"""select var_pop(gamePointId)  as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_130
-  test("Queries_DICTIONARY_EXCLUDE_130", Include) {
-
-    sql(s"""select var_samp(gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_131
-  test("Queries_DICTIONARY_EXCLUDE_131", Include) {
-
-    sql(s"""select stddev_pop(gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_132
-  test("Queries_DICTIONARY_EXCLUDE_132", Include) {
-
-    sql(s"""select stddev_samp(gamePointId)  as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_133
-  test("Queries_DICTIONARY_EXCLUDE_133", Include) {
-
-    sql(s"""select covar_pop(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_134
-  test("Queries_DICTIONARY_EXCLUDE_134", Include) {
-
-    sql(s"""select covar_samp(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_135
-  test("Queries_DICTIONARY_EXCLUDE_135", Include) {
-
-    checkAnswer(s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_135")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_136
-  test("Queries_DICTIONARY_EXCLUDE_136", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_137
-  test("Queries_DICTIONARY_EXCLUDE_137", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2,5) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_138
-  test("Queries_DICTIONARY_EXCLUDE_138", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99))  as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_139
-  test("Queries_DICTIONARY_EXCLUDE_139", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99),5) as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_140
-  test("Queries_DICTIONARY_EXCLUDE_140", Include) {
-
-    sql(s"""select histogram_numeric(gamePointId,2)  as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_141
-  test("Queries_DICTIONARY_EXCLUDE_141", Include) {
-
-    checkAnswer(s"""select gamePointId, gamePointId+ 10 as a  from TABLE_DICTIONARY_EXCLUDE order by a""",
-      s"""select gamePointId, gamePointId+ 10 as a  from TABLE_DICTIONARY_EXCLUDE1_hive order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_141")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_142
-  test("Queries_DICTIONARY_EXCLUDE_142", Include) {
-
-    checkAnswer(s"""select min(gamePointId), max(gamePointId+ 10) Total from TABLE_DICTIONARY_EXCLUDE group by  channelsId order by Total""",
-      s"""select min(gamePointId), max(gamePointId+ 10) Total from TABLE_DICTIONARY_EXCLUDE1_hive group by  channelsId order by Total""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_142")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_143
-  test("Queries_DICTIONARY_EXCLUDE_143", Include) {
-
-    sql(s"""select last(gamePointId) a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_144
-  ignore("Queries_DICTIONARY_EXCLUDE_144", Include) {
-
-    checkAnswer(s"""select FIRST(gamePointId) a from TABLE_DICTIONARY_EXCLUDE order by a""",
-      s"""select FIRST(gamePointId) a from TABLE_DICTIONARY_EXCLUDE1_hive order by a""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_144")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_145
-  test("Queries_DICTIONARY_EXCLUDE_145", Include) {
-
-    checkAnswer(s"""select gamePointId,count(gamePointId) a from TABLE_DICTIONARY_EXCLUDE group by gamePointId order by gamePointId""",
-      s"""select gamePointId,count(gamePointId) a from TABLE_DICTIONARY_EXCLUDE1_hive group by gamePointId order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_145")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_146
-  test("Queries_DICTIONARY_EXCLUDE_146", Include) {
-
-    checkAnswer(s"""select Lower(gamePointId) a  from TABLE_DICTIONARY_EXCLUDE order by gamePointId""",
-      s"""select Lower(gamePointId) a  from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_146")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_147
-  test("Queries_DICTIONARY_EXCLUDE_147", Include) {
-
-    checkAnswer(s"""select distinct gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId""",
-      s"""select distinct gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_147")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_148
-  test("Queries_DICTIONARY_EXCLUDE_148", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE  order by gamePointId limit 101""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive  order by gamePointId limit 101""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_148")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_149
-  test("Queries_DICTIONARY_EXCLUDE_149", Include) {
-
-    checkAnswer(s"""select gamePointId as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""",
-      s"""select gamePointId as a from TABLE_DICTIONARY_EXCLUDE1_hive  order by a asc limit 10""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_149")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_150
-  test("Queries_DICTIONARY_EXCLUDE_150", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where  (gamePointId == 4.70133553923674E43) and (imei=='1AA100084')""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where  (gamePointId == 4.70133553923674E43) and (imei=='1AA100084')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_150")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_151
-  test("Queries_DICTIONARY_EXCLUDE_151", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId !=4.70133553923674E43  order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId !=4.70133553923674E43  order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_151")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_152
-  test("Queries_DICTIONARY_EXCLUDE_152", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_152")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_153
-  test("Queries_DICTIONARY_EXCLUDE_153", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId !=4.70133553923674E43""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId !=4.70133553923674E43""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_153")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_154
-  test("Queries_DICTIONARY_EXCLUDE_154", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId >4.70133553923674E43""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId >4.70133553923674E43""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_154")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_155
-  test("Queries_DICTIONARY_EXCLUDE_155", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE where gamePointId<>gamePointId""",
-      s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId<>gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_155")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_156
-  test("Queries_DICTIONARY_EXCLUDE_156", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId != Latest_areaId  order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId != Latest_areaId  order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_156")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_157
-  test("Queries_DICTIONARY_EXCLUDE_157", Include) {
-
-    checkAnswer(s"""select gamePointId, gamePointId from TABLE_DICTIONARY_EXCLUDE where Latest_areaId<gamePointId  order by gamePointId""",
-      s"""select gamePointId, gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_areaId<gamePointId  order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_157")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_158
-  test("Queries_DICTIONARY_EXCLUDE_158", Include) {
-
-    checkAnswer(s"""select gamePointId, gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId<=gamePointId  order by gamePointId""",
-      s"""select gamePointId, gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId<=gamePointId  order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_158")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_159
-  test("Queries_DICTIONARY_EXCLUDE_159", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId <1000 order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId <1000 order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_159")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_160
-  test("Queries_DICTIONARY_EXCLUDE_160", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId >1000 order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId >1000 order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_160")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_161
-  test("Queries_DICTIONARY_EXCLUDE_161", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE where gamePointId IS NULL order by gamePointId""",
-      s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId IS NULL order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_161")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_162
-  test("Queries_DICTIONARY_EXCLUDE_162", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE where gamePointId IS NOT NULL order by gamePointId""",
-      s"""select gamePointId  from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId IS NOT NULL order by gamePointId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_162")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_163
-  test("Queries_DICTIONARY_EXCLUDE_163", Include) {
-
-    checkAnswer(s"""Select count(productionDate) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""Select count(productionDate) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_163")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_164
-  test("Queries_DICTIONARY_EXCLUDE_164", Include) {
-
-    checkAnswer(s"""select count(DISTINCT productionDate) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT productionDate) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_164")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_165
-  test("Queries_DICTIONARY_EXCLUDE_165", Include) {
-
-    checkAnswer(s"""select sum(productionDate)+10 as a ,productionDate  from TABLE_DICTIONARY_EXCLUDE group by productionDate order by productionDate""",
-      s"""select sum(productionDate)+10 as a ,productionDate  from TABLE_DICTIONARY_EXCLUDE1_hive group by productionDate order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_165")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_166
-  test("Queries_DICTIONARY_EXCLUDE_166", Include) {
-
-    checkAnswer(s"""select max(productionDate),min(productionDate) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(productionDate),min(productionDate) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_166")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_167
-  test("Queries_DICTIONARY_EXCLUDE_167", Include) {
-
-    checkAnswer(s"""select sum(productionDate) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(productionDate) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_167")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_168
-  test("Queries_DICTIONARY_EXCLUDE_168", Include) {
-
-    checkAnswer(s"""select avg(productionDate) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select avg(productionDate) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_168")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_169
-  test("Queries_DICTIONARY_EXCLUDE_169", Include) {
-
-    checkAnswer(s"""select min(productionDate) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select min(productionDate) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_169")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_170
-  test("Queries_DICTIONARY_EXCLUDE_170", Include) {
-
-    sql(s"""select variance(gamePointId) as a   from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_171
-  ignore("Queries_DICTIONARY_EXCLUDE_171", Include) {
-
-    checkAnswer(s"""select var_pop(gamePointId) as a from (select * from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select var_pop(gamePointId) as a from (select * from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_171")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_172
-  ignore("Queries_DICTIONARY_EXCLUDE_172", Include) {
-
-    checkAnswer(s"""select var_samp(gamePointId) as a from (select * from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select var_samp(gamePointId) as a from (select * from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_172")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_173
-  test("Queries_DICTIONARY_EXCLUDE_173", Include) {
-
-    sql(s"""select stddev_pop(gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_174
-  test("Queries_DICTIONARY_EXCLUDE_174", Include) {
-
-    sql(s"""select stddev_samp(gamePointId)  as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_175
-  test("Queries_DICTIONARY_EXCLUDE_175", Include) {
-
-    sql(s"""select covar_pop(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_176
-  test("Queries_DICTIONARY_EXCLUDE_176", Include) {
-
-    sql(s"""select covar_samp(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_177
-  test("Queries_DICTIONARY_EXCLUDE_177", Include) {
-
-    checkAnswer(s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_177")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_178
-  test("Queries_DICTIONARY_EXCLUDE_178", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_179
-  test("Queries_DICTIONARY_EXCLUDE_179", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2,5) as a  from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_180
-  test("Queries_DICTIONARY_EXCLUDE_180", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99))  as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_181
-  test("Queries_DICTIONARY_EXCLUDE_181", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99),5) as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_182
-  test("Queries_DICTIONARY_EXCLUDE_182", Include) {
-
-    sql(s"""select histogram_numeric(productionDate,2)  as a from (select productionDate from TABLE_DICTIONARY_EXCLUDE order by productionDate) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_183
-  test("Queries_DICTIONARY_EXCLUDE_183", Include) {
-
-    sql(s"""select last(productionDate) a from TABLE_DICTIONARY_EXCLUDE order by a""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_184
-  test("Queries_DICTIONARY_EXCLUDE_184", Include) {
-
-    sql(s"""select FIRST(productionDate) a from (select productionDate from TABLE_DICTIONARY_EXCLUDE order by productionDate) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_185
-  test("Queries_DICTIONARY_EXCLUDE_185", Include) {
-
-    checkAnswer(s"""select productionDate,count(productionDate) a from TABLE_DICTIONARY_EXCLUDE group by productionDate order by productionDate""",
-      s"""select productionDate,count(productionDate) a from TABLE_DICTIONARY_EXCLUDE1_hive group by productionDate order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_185")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_186
-  test("Queries_DICTIONARY_EXCLUDE_186", Include) {
-
-    checkAnswer(s"""select Lower(productionDate) a  from TABLE_DICTIONARY_EXCLUDE order by productionDate""",
-      s"""select Lower(productionDate) a  from TABLE_DICTIONARY_EXCLUDE1_hive order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_186")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_187
-  test("Queries_DICTIONARY_EXCLUDE_187", Include) {
-
-    checkAnswer(s"""select distinct productionDate from TABLE_DICTIONARY_EXCLUDE order by productionDate""",
-      s"""select distinct productionDate from TABLE_DICTIONARY_EXCLUDE1_hive order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_187")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_188
-  test("Queries_DICTIONARY_EXCLUDE_188", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE order by productionDate limit 101""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive order by productionDate limit 101""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_188")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_189
-  test("Queries_DICTIONARY_EXCLUDE_189", Include) {
-
-    checkAnswer(s"""select productionDate as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""",
-      s"""select productionDate as a from TABLE_DICTIONARY_EXCLUDE1_hive  order by a asc limit 10""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_189")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_190
-  test("Queries_DICTIONARY_EXCLUDE_190", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where  (productionDate == '2015-07-01 12:07:28.0') and (productionDate=='2015-07-01 12:07:28.0')""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where  (productionDate == '2015-07-01 12:07:28.0') and (productionDate=='2015-07-01 12:07:28.0')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_190")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_191
-  test("Queries_DICTIONARY_EXCLUDE_191", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate !='2015-07-01 12:07:28.0' order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate !='2015-07-01 12:07:28.0' order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_191")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_192
-  test("Queries_DICTIONARY_EXCLUDE_192", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_192")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_193
-  test("Queries_DICTIONARY_EXCLUDE_193", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate !='2015-07-01 12:07:28.0' order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate !='2015-07-01 12:07:28.0' order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_193")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_194
-  test("Queries_DICTIONARY_EXCLUDE_194", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate >'2015-07-01 12:07:28.0' order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate >'2015-07-01 12:07:28.0' order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_194")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_195
-  test("Queries_DICTIONARY_EXCLUDE_195", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE where productionDate<>productionDate order by productionDate""",
-      s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate<>productionDate order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_195")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_196
-  test("Queries_DICTIONARY_EXCLUDE_196", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate != Latest_areaId order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate != Latest_areaId order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_196")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_197
-  test("Queries_DICTIONARY_EXCLUDE_197", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where Latest_areaId<productionDate order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_areaId<productionDate order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_197")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_198
-  test("Queries_DICTIONARY_EXCLUDE_198", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate<=productionDate order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate<=productionDate order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_198")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_199
-  test("Queries_DICTIONARY_EXCLUDE_199", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate <cast('2015-07-01 12:07:28.0' as timestamp) order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate <cast('2015-07-01 12:07:28.0' as timestamp) order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_199")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_200
-  test("Queries_DICTIONARY_EXCLUDE_200", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE where productionDate IS NULL""",
-      s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate IS NULL""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_200")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_201
-  test("Queries_DICTIONARY_EXCLUDE_201", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE where productionDate IS NOT NULL order by productionDate""",
-      s"""select productionDate  from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate IS NOT NULL order by productionDate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_201")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_202
-  test("Queries_DICTIONARY_EXCLUDE_202", Include) {
-
-    checkAnswer(s"""Select count(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""Select count(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_202")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_203
-  test("Queries_DICTIONARY_EXCLUDE_203", Include) {
-
-    checkAnswer(s"""select count(DISTINCT deviceInformationId) as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(DISTINCT deviceInformationId) as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_203")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_204
-  test("Queries_DICTIONARY_EXCLUDE_204", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)+10 as a ,deviceInformationId  from TABLE_DICTIONARY_EXCLUDE group by deviceInformationId order by deviceInformationId""",
-      s"""select sum(deviceInformationId)+10 as a ,deviceInformationId  from TABLE_DICTIONARY_EXCLUDE1_hive group by deviceInformationId order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_204")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_205
-  test("Queries_DICTIONARY_EXCLUDE_205", Include) {
-
-    checkAnswer(s"""select max(deviceInformationId),min(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select max(deviceInformationId),min(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_205")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_206
-  test("Queries_DICTIONARY_EXCLUDE_206", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_206")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_207
-  test("Queries_DICTIONARY_EXCLUDE_207", Include) {
-
-    checkAnswer(s"""select avg(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select avg(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_207")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_208
-  test("Queries_DICTIONARY_EXCLUDE_208", Include) {
-
-    checkAnswer(s"""select min(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select min(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_208")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_209
-  test("Queries_DICTIONARY_EXCLUDE_209", Include) {
-
-    sql(s"""select variance(deviceInformationId) as a   from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_210
-  ignore("Queries_DICTIONARY_EXCLUDE_210", Include) {
-
-    checkAnswer(s"""select var_pop(deviceInformationId)  as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select var_pop(deviceInformationId)  as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_210")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_211
-  ignore("Queries_DICTIONARY_EXCLUDE_211", Include) {
-
-    checkAnswer(s"""select var_samp(deviceInformationId) as a  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select var_samp(deviceInformationId) as a  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_211")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_212
-  test("Queries_DICTIONARY_EXCLUDE_212", Include) {
-
-    sql(s"""select stddev_pop(deviceInformationId) as a  from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_213
-  test("Queries_DICTIONARY_EXCLUDE_213", Include) {
-
-    sql(s"""select stddev_samp(deviceInformationId)  as a from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_214
-  test("Queries_DICTIONARY_EXCLUDE_214", Include) {
-
-    sql(s"""select covar_pop(deviceInformationId,deviceInformationId) as a  from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_215
-  test("Queries_DICTIONARY_EXCLUDE_215", Include) {
-
-    sql(s"""select covar_samp(deviceInformationId,deviceInformationId) as a  from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_216
-  test("Queries_DICTIONARY_EXCLUDE_216", Include) {
-
-    checkAnswer(s"""select corr(deviceInformationId,deviceInformationId)  as a from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select corr(deviceInformationId,deviceInformationId)  as a from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_216")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_217
-  test("Queries_DICTIONARY_EXCLUDE_217", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,0.2) as a  from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_218
-  test("Queries_DICTIONARY_EXCLUDE_218", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,0.2,5) as a  from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_219
-  test("Queries_DICTIONARY_EXCLUDE_219", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,array(0.2,0.3,0.99))  as a from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_220
-  test("Queries_DICTIONARY_EXCLUDE_220", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,array(0.2,0.3,0.99),5) as a from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_221
-  test("Queries_DICTIONARY_EXCLUDE_221", Include) {
-
-    sql(s"""select histogram_numeric(deviceInformationId,2)  as a from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_222
-  test("Queries_DICTIONARY_EXCLUDE_222", Include) {
-
-    checkAnswer(s"""select deviceInformationId, deviceInformationId+ 10 as a  from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId""",
-      s"""select deviceInformationId, deviceInformationId+ 10 as a  from TABLE_DICTIONARY_EXCLUDE1_hive order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_222")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_223
-  test("Queries_DICTIONARY_EXCLUDE_223", Include) {
-
-    checkAnswer(s"""select min(deviceInformationId), max(deviceInformationId+ 10) Total from TABLE_DICTIONARY_EXCLUDE group by  channelsId order by Total""",
-      s"""select min(deviceInformationId), max(deviceInformationId+ 10) Total from TABLE_DICTIONARY_EXCLUDE1_hive group by  channelsId order by Total""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_223")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_224
-  test("Queries_DICTIONARY_EXCLUDE_224", Include) {
-
-    sql(s"""select last(deviceInformationId) a from TABLE_DICTIONARY_EXCLUDE order by a""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_225
-  test("Queries_DICTIONARY_EXCLUDE_225", Include) {
-
-    sql(s"""select FIRST(deviceInformationId) a from (select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_226
-  test("Queries_DICTIONARY_EXCLUDE_226", Include) {
-
-    checkAnswer(s"""select deviceInformationId,count(deviceInformationId) a from TABLE_DICTIONARY_EXCLUDE group by deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId,count(deviceInformationId) a from TABLE_DICTIONARY_EXCLUDE1_hive group by deviceInformationId order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_226")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_227
-  test("Queries_DICTIONARY_EXCLUDE_227", Include) {
-
-    checkAnswer(s"""select Lower(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId""",
-      s"""select Lower(deviceInformationId) a  from TABLE_DICTIONARY_EXCLUDE1_hive order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_227")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_228
-  test("Queries_DICTIONARY_EXCLUDE_228", Include) {
-
-    checkAnswer(s"""select distinct deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId""",
-      s"""select distinct deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_228")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_229
-  test("Queries_DICTIONARY_EXCLUDE_229", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE order by deviceInformationId limit 101""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive order by deviceInformationId limit 101""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_229")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_230
-  test("Queries_DICTIONARY_EXCLUDE_230", Include) {
-
-    checkAnswer(s"""select deviceInformationId as a from TABLE_DICTIONARY_EXCLUDE  order by a asc limit 10""",
-      s"""select deviceInformationId as a from TABLE_DICTIONARY_EXCLUDE1_hive  order by a asc limit 10""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_230")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_231
-  test("Queries_DICTIONARY_EXCLUDE_231", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where  (deviceInformationId == 100084) and (deviceInformationId==100084)""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where  (deviceInformationId == 100084) and (deviceInformationId==100084)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_231")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_232
-  test("Queries_DICTIONARY_EXCLUDE_232", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId !='100084' order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId !='100084' order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_232")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_233
-  test("Queries_DICTIONARY_EXCLUDE_233", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE1_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_233")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_234
-  test("Queries_DICTIONARY_EXCLUDE_234", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId !=100084 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId !=100084 order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_234")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_235
-  test("Queries_DICTIONARY_EXCLUDE_235", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId >100084 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId >100084 order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_235")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_236
-  test("Queries_DICTIONARY_EXCLUDE_236", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE where deviceInformationId<>deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId<>deviceInformationId order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_236")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_237
-  test("Queries_DICTIONARY_EXCLUDE_237", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId != Latest_areaId order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId != Latest_areaId order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_237")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_238
-  test("Queries_DICTIONARY_EXCLUDE_238", Include) {
-
-    checkAnswer(s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_EXCLUDE where Latest_areaId<deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_areaId<deviceInformationId order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_238")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_239
-  test("Queries_DICTIONARY_EXCLUDE_239", Include) {
-
-    checkAnswer(s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId<=deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId<=deviceInformationId order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_239")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_240
-  test("Queries_DICTIONARY_EXCLUDE_240", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId <1000 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId <1000 order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_240")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_241
-  test("Queries_DICTIONARY_EXCLUDE_241", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId >1000 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId >1000 order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_241")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_242
-  test("Queries_DICTIONARY_EXCLUDE_242", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE where deviceInformationId IS NULL order by deviceInformationId""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId IS NULL order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_242")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_243
-  test("Queries_DICTIONARY_EXCLUDE_243", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE where deviceInformationId IS NOT NULL order by deviceInformationId""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId IS NOT NULL order by deviceInformationId""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_243")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_244
-  test("Queries_DICTIONARY_EXCLUDE_244", Include) {
-
-    checkAnswer(s"""select sum(imei)+10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(imei)+10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_244")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_245
-  test("Queries_DICTIONARY_EXCLUDE_245", Include) {
-
-    checkAnswer(s"""select sum(imei)*10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(imei)*10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_245")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_246
-  test("Queries_DICTIONARY_EXCLUDE_246", Include) {
-
-    checkAnswer(s"""select sum(imei)/10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(imei)/10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_246")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_247
-  test("Queries_DICTIONARY_EXCLUDE_247", Include) {
-
-    checkAnswer(s"""select sum(imei)-10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(imei)-10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_247")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_248
-  test("Queries_DICTIONARY_EXCLUDE_248", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)+10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(contractNumber)+10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_248")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_249
-  test("Queries_DICTIONARY_EXCLUDE_249", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)*10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(contractNumber)*10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_249")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_250
-  test("Queries_DICTIONARY_EXCLUDE_250", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)/10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(contractNumber)/10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_250")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_251
-  test("Queries_DICTIONARY_EXCLUDE_251", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)-10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(contractNumber)-10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_251")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_252
-  test("Queries_DICTIONARY_EXCLUDE_252", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)+10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(Latest_DAY)+10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_252")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_253
-  test("Queries_DICTIONARY_EXCLUDE_253", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)*10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(Latest_DAY)*10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_253")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_254
-  test("Queries_DICTIONARY_EXCLUDE_254", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)/10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(Latest_DAY)/10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_254")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_255
-  test("Queries_DICTIONARY_EXCLUDE_255", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)-10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(Latest_DAY)-10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_255")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_256
-  ignore("Queries_DICTIONARY_EXCLUDE_256", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)+10 as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select sum(gamePointId)+10 as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_256")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_257
-  ignore("Queries_DICTIONARY_EXCLUDE_257", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)*10 as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select sum(gamePointId)*10 as a from (select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_257")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_258
-  test("Queries_DICTIONARY_EXCLUDE_258", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)/10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(gamePointId)/10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_258")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_259
-  ignore("Queries_DICTIONARY_EXCLUDE_259", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)-10 as a   from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select sum(gamePointId)-10 as a   from (select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_259")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_260
-  test("Queries_DICTIONARY_EXCLUDE_260", Include) {
-
-    checkAnswer(s"""select sum(productionDate)+10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(productionDate)+10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_260")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_261
-  test("Queries_DICTIONARY_EXCLUDE_261", Include) {
-
-    checkAnswer(s"""select sum(productionDate)*10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(productionDate)*10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_261")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_262
-  test("Queries_DICTIONARY_EXCLUDE_262", Include) {
-
-    checkAnswer(s"""select sum(productionDate)/10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(productionDate)/10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_262")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_263
-  test("Queries_DICTIONARY_EXCLUDE_263", Include) {
-
-    checkAnswer(s"""select sum(productionDate)-10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(productionDate)-10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_263")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_264
-  test("Queries_DICTIONARY_EXCLUDE_264", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)+10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(deviceInformationId)+10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_264")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_265
-  test("Queries_DICTIONARY_EXCLUDE_265", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)*10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(deviceInformationId)*10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_265")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_266
-  test("Queries_DICTIONARY_EXCLUDE_266", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)/10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(deviceInformationId)/10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_266")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_267
-  test("Queries_DICTIONARY_EXCLUDE_267", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)-10 as a   from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(deviceInformationId)-10 as a   from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_267")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_292
-  test("Queries_DICTIONARY_EXCLUDE_292", Include) {
-
-    checkAnswer(s"""SELECT productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate LIKE '2015-09-30%'""",
-      s"""SELECT productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate LIKE '2015-09-30%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_292")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_293
-  test("Queries_DICTIONARY_EXCLUDE_293", Include) {
-
-    checkAnswer(s"""SELECT productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate LIKE '% %'""",
-      s"""SELECT productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate LIKE '% %'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_293")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_294
-  test("Queries_DICTIONARY_EXCLUDE_294", Include) {
-
-    checkAnswer(s"""SELECT productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate LIKE '%12:07:28'""",
-      s"""SELECT productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate LIKE '%12:07:28'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_294")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_295
-  test("Queries_DICTIONARY_EXCLUDE_295", Include) {
-
-    checkAnswer(s"""select contractnumber from TABLE_DICTIONARY_EXCLUDE where contractnumber like '922337204%' """,
-      s"""select contractnumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractnumber like '922337204%' """, "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_295")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_296
-  test("Queries_DICTIONARY_EXCLUDE_296", Include) {
-
-    checkAnswer(s"""select contractnumber from TABLE_DICTIONARY_EXCLUDE where contractnumber like '%047800'""",
-      s"""select contractnumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractnumber like '%047800'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_296")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_297
-  test("Queries_DICTIONARY_EXCLUDE_297", Include) {
-
-    checkAnswer(s"""select contractnumber from TABLE_DICTIONARY_EXCLUDE where contractnumber like '%720%'""",
-      s"""select contractnumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractnumber like '%720%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_297")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_298
-  test("Queries_DICTIONARY_EXCLUDE_298", Include) {
-
-    checkAnswer(s"""SELECT Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY like '12345678%'""",
-      s"""SELECT Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY like '12345678%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_298")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_299
-  test("Queries_DICTIONARY_EXCLUDE_299", Include) {
-
-    checkAnswer(s"""SELECT Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY like '%5678%'""",
-      s"""SELECT Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY like '%5678%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_299")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_300
-  test("Queries_DICTIONARY_EXCLUDE_300", Include) {
-
-    checkAnswer(s"""SELECT Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY like '1234567%'""",
-      s"""SELECT Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY like '1234567%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_300")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_301
-  test("Queries_DICTIONARY_EXCLUDE_301", Include) {
-
-    checkAnswer(s"""SELECT gamepointID from TABLE_DICTIONARY_EXCLUDE where gamepointID like '1.1098347722%'""",
-      s"""SELECT gamepointID from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointID like '1.1098347722%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_301")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_302
-  test("Queries_DICTIONARY_EXCLUDE_302", Include) {
-
-    checkAnswer(s"""SELECT gamepointID from TABLE_DICTIONARY_EXCLUDE where gamepointID like '%8347722%'""",
-      s"""SELECT gamepointID from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointID like '%8347722%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_302")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_303
-  test("Queries_DICTIONARY_EXCLUDE_303", Include) {
-
-    checkAnswer(s"""SELECT gamepointID from TABLE_DICTIONARY_EXCLUDE where gamepointID like '%7512E42'""",
-      s"""SELECT gamepointID from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointID like '%7512E42'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_303")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_304
-  test("Queries_DICTIONARY_EXCLUDE_304", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid like '1000%'""",
-      s"""SELECT deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid like '1000%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_304")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_305
-  test("Queries_DICTIONARY_EXCLUDE_305", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid like '%00%'""",
-      s"""SELECT deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid like '%00%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_305")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_306
-  test("Queries_DICTIONARY_EXCLUDE_306", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid like '%0084'""",
-      s"""SELECT deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid like '%0084'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_306")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_307
-  test("Queries_DICTIONARY_EXCLUDE_307", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei like '1AA10%'""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei like '1AA10%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_307")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_308
-  test("Queries_DICTIONARY_EXCLUDE_308", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei like '%A10%'""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei like '%A10%'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_308")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_309
-  test("Queries_DICTIONARY_EXCLUDE_309", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei like '%00084'""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei like '%00084'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_309")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_310
-  test("Queries_DICTIONARY_EXCLUDE_310", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei in ('1AA100074','1AA100075','1AA100077')""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei in ('1AA100074','1AA100075','1AA100077')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_310")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_311
-  test("Queries_DICTIONARY_EXCLUDE_311", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei not in ('1AA100074','1AA100075','1AA100077')""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei not in ('1AA100074','1AA100075','1AA100077')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_311")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_312
-  test("Queries_DICTIONARY_EXCLUDE_312", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid in (100081,100078,10008)""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid in (100081,100078,10008)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_312")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_313
-  test("Queries_DICTIONARY_EXCLUDE_313", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid not in (100081,100078,10008)""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid not in (100081,100078,10008)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_313")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_314
-  test("Queries_DICTIONARY_EXCLUDE_314", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate in ('2015-10-04 12:07:28','2015-10-07%','2015-10-07 12:07:28')""",
-      s"""select productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate in ('2015-10-04 12:07:28','2015-10-07%','2015-10-07 12:07:28')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_314")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_315
-  test("Queries_DICTIONARY_EXCLUDE_315", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate not in (cast('2015-10-04 12:07:28' as timestamp),cast('2015-10-07 12:07:28' as timestamp))""",
-      s"""select productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate not in (cast('2015-10-04 12:07:28' as timestamp),cast('2015-10-07 12:07:28' as timestamp))""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_315")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_316
-  test("Queries_DICTIONARY_EXCLUDE_316", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE where gamepointid in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""",
-      s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointid in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_316")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_317
-  test("Queries_DICTIONARY_EXCLUDE_317", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE where gamepointid not in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""",
-      s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointid not in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_317")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_318
-  test("Queries_DICTIONARY_EXCLUDE_318", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY in (1234567890123530.0000000000,1234567890123520.0000000000)""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY in (1234567890123530.0000000000,1234567890123520.0000000000)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_318")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_319
-  test("Queries_DICTIONARY_EXCLUDE_319", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY not in (1234567890123530.0000000000,1234567890123520.0000000000)""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY not in (1234567890123530.0000000000,1234567890123520.0000000000)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_319")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_322
-  test("Queries_DICTIONARY_EXCLUDE_322", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei !='1AA100077'""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei !='1AA100077'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_322")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_323
-  test("Queries_DICTIONARY_EXCLUDE_323", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei NOT LIKE '1AA100077'""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei NOT LIKE '1AA100077'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_323")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_324
-  test("Queries_DICTIONARY_EXCLUDE_324", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid !=100078""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid !=100078""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_324")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_325
-  test("Queries_DICTIONARY_EXCLUDE_325", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE where deviceinformationid NOT LIKE 100079""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationid NOT LIKE 100079""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_325")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_326
-  test("Queries_DICTIONARY_EXCLUDE_326", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate !='2015-10-07 12:07:28'""",
-      s"""select productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate !='2015-10-07 12:07:28'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_326")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_327
-  ignore("Queries_DICTIONARY_EXCLUDE_327", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_EXCLUDE where productiondate NOT LIKE '2015-10-07 12:07:28'""",
-      s"""select productiondate from TABLE_DICTIONARY_EXCLUDE1_hive where productiondate NOT LIKE '2015-10-07 12:07:28'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_327")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_328
-  test("Queries_DICTIONARY_EXCLUDE_328", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE where gamepointid !=6.8591561117512E42""",
-      s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointid !=6.8591561117512E42""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_328")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_329
-  test("Queries_DICTIONARY_EXCLUDE_329", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE where gamepointid NOT LIKE 6.8591561117512E43""",
-      s"""select gamepointid from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointid NOT LIKE 6.8591561117512E43""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_329")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_330
-  test("Queries_DICTIONARY_EXCLUDE_330", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY != 1234567890123520.0000000000""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY != 1234567890123520.0000000000""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_330")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_331
-  test("Queries_DICTIONARY_EXCLUDE_331", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY NOT LIKE 1234567890123520.0000000000""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY NOT LIKE 1234567890123520.0000000000""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_331")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_335
-  test("Queries_DICTIONARY_EXCLUDE_335", Include) {
-
-    checkAnswer(s"""SELECT productiondate,IMEI from TABLE_DICTIONARY_EXCLUDE where IMEI RLIKE '1AA100077'""",
-      s"""SELECT productiondate,IMEI from TABLE_DICTIONARY_EXCLUDE1_hive where IMEI RLIKE '1AA100077'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_335")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_336
-  test("Queries_DICTIONARY_EXCLUDE_336", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationId from TABLE_DICTIONARY_EXCLUDE where deviceinformationId RLIKE '100079'""",
-      s"""SELECT deviceinformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceinformationId RLIKE '100079'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_336")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_337
-  test("Queries_DICTIONARY_EXCLUDE_337", Include) {
-
-    checkAnswer(s"""SELECT gamepointid from TABLE_DICTIONARY_EXCLUDE where gamepointid RLIKE '1.61922711065643E42'""",
-      s"""SELECT gamepointid from TABLE_DICTIONARY_EXCLUDE1_hive where gamepointid RLIKE '1.61922711065643E42'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_337")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_338
-  test("Queries_DICTIONARY_EXCLUDE_338", Include) {
-
-    checkAnswer(s"""SELECT Latest_Day from TABLE_DICTIONARY_EXCLUDE where Latest_Day RLIKE '1234567890123550.0000000000'""",
-      s"""SELECT Latest_Day from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_Day RLIKE '1234567890123550.0000000000'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_338")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_339
-  test("Queries_DICTIONARY_EXCLUDE_339", Include) {
-
-    checkAnswer(s"""SELECT contractnumber from TABLE_DICTIONARY_EXCLUDE where contractnumber RLIKE '9223372047800'""",
-      s"""SELECT contractnumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractnumber RLIKE '9223372047800'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_339")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_340
-  test("Queries_DICTIONARY_EXCLUDE_340", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE a join TABLE_DICTIONARY_EXCLUDE b on a.productiondate=b.productiondate""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE1_hive a join TABLE_DICTIONARY_EXCLUDE1_hive b on a.productiondate=b.productiondate""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_340")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_341
-  test("Queries_DICTIONARY_EXCLUDE_341", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE a join TABLE_DICTIONARY_EXCLUDE b on a.deviceinformationid=b.deviceinformationid""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE1_hive a join TABLE_DICTIONARY_EXCLUDE1_hive b on a.deviceinformationid=b.deviceinformationid""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_341")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_342
-  test("Queries_DICTIONARY_EXCLUDE_342", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE a join TABLE_DICTIONARY_EXCLUDE b on a.imei=b.imei""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE1_hive a join TABLE_DICTIONARY_EXCLUDE1_hive b on a.imei=b.imei""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_342")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_343
-  test("Queries_DICTIONARY_EXCLUDE_343", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE a join TABLE_DICTIONARY_EXCLUDE b on a.gamepointid=b.gamepointid""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE1_hive a join TABLE_DICTIONARY_EXCLUDE1_hive b on a.gamepointid=b.gamepointid""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_343")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_344
-  test("Queries_DICTIONARY_EXCLUDE_344", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE a join TABLE_DICTIONARY_EXCLUDE b on a.Latest_Day=b.Latest_Day""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE1_hive a join TABLE_DICTIONARY_EXCLUDE1_hive b on a.Latest_Day=b.Latest_Day""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_344")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_345
-  test("Queries_DICTIONARY_EXCLUDE_345", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE a join TABLE_DICTIONARY_EXCLUDE b on a.contractnumber=b.contractnumber""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_EXCLUDE1_hive a join TABLE_DICTIONARY_EXCLUDE1_hive b on a.contractnumber=b.contractnumber""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_345")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_346
-  test("Queries_DICTIONARY_EXCLUDE_346", Include) {
-
-    checkAnswer(s"""select count( contractNumber ),sum( contractNumber ),count(distinct contractNumber ),avg( contractNumber ),max( contractNumber ),min( contractNumber ),1 from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count( contractNumber ),sum( contractNumber ),count(distinct contractNumber ),avg( contractNumber ),max( contractNumber ),min( contractNumber ),1 from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_346")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_347
-  test("Queries_DICTIONARY_EXCLUDE_347", Include) {
-
-    checkAnswer(s"""select count( Latest_Day ),sum( Latest_Day ),count(distinct Latest_Day ),avg( Latest_Day ),max( Latest_Day ),min( Latest_Day ),1 from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count( Latest_Day ),sum( Latest_Day ),count(distinct Latest_Day ),avg( Latest_Day ),max( Latest_Day ),min( Latest_Day ),1 from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_347")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_348
-  ignore("Queries_DICTIONARY_EXCLUDE_348", Include) {
-
-    checkAnswer(s"""select count( gamePointId),sum( gamePointId ),count(distinct gamePointId ),avg(gamePointId),max(gamePointId),min(gamePointId),1 from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count( gamePointId),sum( gamePointId ),count(distinct gamePointId ),avg(gamePointId),max(gamePointId),min(gamePointId),1 from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_348")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_349
-  test("Queries_DICTIONARY_EXCLUDE_349", Include) {
-
-    checkAnswer(s"""select count(deviceInformationId),sum(deviceInformationId),count(deviceInformationId),avg(deviceInformationId),max(deviceInformationId),min(deviceInformationId),1 from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(deviceInformationId),sum(deviceInformationId),count(deviceInformationId),avg(deviceInformationId),max(deviceInformationId),min(deviceInformationId),1 from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_349")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_350
-  test("Queries_DICTIONARY_EXCLUDE_350", Include) {
-
-    checkAnswer(s"""select count( productionDate),sum(  productionDate ),count(distinct  productionDate ),avg(  productionDate ),max(  productionDate),min(  productionDate ),1 from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count( productionDate),sum(  productionDate ),count(distinct  productionDate ),avg(  productionDate ),max(  productionDate),min(  productionDate ),1 from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_350")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_351
-  test("Queries_DICTIONARY_EXCLUDE_351", Include) {
-
-    checkAnswer(s"""select count(IMEI ),sum(IMEI ),count(distinct IMEI ),avg(IMEI ),max(IMEI ),min(IMEI ),1 from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(IMEI ),sum(IMEI ),count(distinct IMEI ),avg(IMEI ),max(IMEI ),min(IMEI ),1 from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_351")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_352
-  test("Queries_DICTIONARY_EXCLUDE_352", Include) {
-
-    checkAnswer(s"""select sum(contractNumber),count(contractNumber),avg(contractNumber),sum(contractNumber)/count(contractNumber) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(contractNumber),count(contractNumber),avg(contractNumber),sum(contractNumber)/count(contractNumber) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_352")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_353
-  test("Queries_DICTIONARY_EXCLUDE_353", Include) {
-
-    checkAnswer(s"""select sum(Latest_Day),count(Latest_Day),avg(Latest_Day),sum(Latest_Day)/count(Latest_Day) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(Latest_Day),count(Latest_Day),avg(Latest_Day),sum(Latest_Day)/count(Latest_Day) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_353")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_354
-  ignore("Queries_DICTIONARY_EXCLUDE_354", Include) {
-
-    checkAnswer(s"""select sum(gamepointId),count(gamepointId),avg(gamepointID),sum(gamepointID)/count(gamepointID) from (select gamePointId from TABLE_DICTIONARY_EXCLUDE order by gamePointId)""",
-      s"""select sum(gamepointId),count(gamepointId),avg(gamepointID),sum(gamepointID)/count(gamepointID) from (select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_354")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_355
-  test("Queries_DICTIONARY_EXCLUDE_355", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId),count(deviceinformationId),avg(deviceinformationId),sum(deviceinformationId)/count(deviceinformationId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(deviceinformationId),count(deviceinformationId),avg(deviceinformationId),sum(deviceinformationId)/count(deviceinformationId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_355")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_356
-  test("Queries_DICTIONARY_EXCLUDE_356", Include) {
-
-    checkAnswer(s"""select sum(IMEI),count(IMEI),avg(IMEI),sum(IMEI)/count(IMEI) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(IMEI),count(IMEI),avg(IMEI),sum(IMEI)/count(IMEI) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_356")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_357
-  test("Queries_DICTIONARY_EXCLUDE_357", Include) {
-
-    checkAnswer(s"""select sum(productionDate),count(productionDate),avg(productionDate),sum(productionDate)/count(productionDate) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select sum(productionDate),count(productionDate),avg(productionDate),sum(productionDate)/count(productionDate) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_357")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_358
-  test("Queries_DICTIONARY_EXCLUDE_358", Include) {
-
-    checkAnswer(s"""select contractNumber,Latest_DAY,gamePointId,productionDate,deviceInformationId,IMEI  from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select contractNumber,Latest_DAY,gamePointId,productionDate,deviceInformationId,IMEI  from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_358")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_359
-  test("Queries_DICTIONARY_EXCLUDE_359", Include) {
-
-    checkAnswer(s"""select count(MAC) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(MAC) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_359")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_360
-  test("Queries_DICTIONARY_EXCLUDE_360", Include) {
-
-    checkAnswer(s"""select count(gamePointId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(gamePointId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_360")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_361
-  test("Queries_DICTIONARY_EXCLUDE_361", Include) {
-
-    checkAnswer(s"""select count(contractNumber) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(contractNumber) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_361")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_362
-  test("Queries_DICTIONARY_EXCLUDE_362", Include) {
-
-    checkAnswer(s"""select count(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_362")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_363
-  test("Queries_DICTIONARY_EXCLUDE_363", Include) {
-
-    checkAnswer(s"""select count(productionDate) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(productionDate) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_363")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_364
-  test("Queries_DICTIONARY_EXCLUDE_364", Include) {
-
-    checkAnswer(s"""select count(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE""",
-      s"""select count(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE1_hive""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_364")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_365
-  test("Queries_DICTIONARY_EXCLUDE_365", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  contractNumber  != '9223372047700'""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  contractNumber  != '9223372047700'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_365")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_366
-  test("Queries_DICTIONARY_EXCLUDE_366", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  Latest_DAY  != '1234567890123480.0000000000' order by imei,deviceInformationId,MAC,deviceColor limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  Latest_DAY  != '1234567890123480.0000000000' order by imei,deviceInformationId,MAC,deviceColor limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_366")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_367
-  test("Queries_DICTIONARY_EXCLUDE_367", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  gamePointId  != '2.27852521808948E36' order by imei,deviceInformationId,MAC,deviceColor limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  gamePointId  != '2.27852521808948E36' order by imei,deviceInformationId,MAC,deviceColor limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_367")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_368
-  test("Queries_DICTIONARY_EXCLUDE_368", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  productionDate  != '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC,deviceColor limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  productionDate  != '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC,deviceColor limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_368")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_369
-  test("Queries_DICTIONARY_EXCLUDE_369", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  deviceInformationId  != 100075 order by imei,deviceInformationId,MAC,deviceColor limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  deviceInformationId  != 100075 order by imei,deviceInformationId,MAC,deviceColor limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_369")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_370
-  test("Queries_DICTIONARY_EXCLUDE_370", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  contractNumber  not like '9223372047700' order by  deviceInformationId limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  contractNumber  not like '9223372047700' order by  deviceInformationId limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_370")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_371
-  test("Queries_DICTIONARY_EXCLUDE_371", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  Latest_DAY  not like '1234567890123480.0000000000' order by imei, deviceInformationId, deviceColor limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  Latest_DAY  not like '1234567890123480.0000000000' order by imei, deviceInformationId, deviceColor limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_371")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_372
-  test("Queries_DICTIONARY_EXCLUDE_372", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  gamePointId  not like '2.27852521808948E36' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  gamePointId  not like '2.27852521808948E36' order by imei,deviceInformationId,MAC limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_372")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_373
-  test("Queries_DICTIONARY_EXCLUDE_373", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  productionDate  not like '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  productionDate  not like '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_373")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_374
-  test("Queries_DICTIONARY_EXCLUDE_374", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE where  deviceInformationId  not like '100075' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_EXCLUDE1_hive where  deviceInformationId  not like '100075' order by imei,deviceInformationId,MAC limit 5""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_374")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_375
-  test("Queries_DICTIONARY_EXCLUDE_375", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei is not null""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei is not null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_375")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_376
-  test("Queries_DICTIONARY_EXCLUDE_376", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId is not null""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId is not null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_376")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_377
-  test("Queries_DICTIONARY_EXCLUDE_377", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber is not null""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber is not null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_377")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_378
-  test("Queries_DICTIONARY_EXCLUDE_378", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY is not null""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY is not null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_378")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_379
-  test("Queries_DICTIONARY_EXCLUDE_379", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate is not null""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate is not null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_379")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_380
-  test("Queries_DICTIONARY_EXCLUDE_380", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId is not null""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId is not null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_380")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_381
-  test("Queries_DICTIONARY_EXCLUDE_381", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_EXCLUDE where imei is  null""",
-      s"""select imei from TABLE_DICTIONARY_EXCLUDE1_hive where imei is  null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_381")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_382
-  test("Queries_DICTIONARY_EXCLUDE_382", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE where gamePointId is  null""",
-      s"""select gamePointId from TABLE_DICTIONARY_EXCLUDE1_hive where gamePointId is  null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_382")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_383
-  test("Queries_DICTIONARY_EXCLUDE_383", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE where contractNumber is  null""",
-      s"""select contractNumber from TABLE_DICTIONARY_EXCLUDE1_hive where contractNumber is  null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_383")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_384
-  test("Queries_DICTIONARY_EXCLUDE_384", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE where Latest_DAY is  null""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_EXCLUDE1_hive where Latest_DAY is  null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_384")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_385
-  test("Queries_DICTIONARY_EXCLUDE_385", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_EXCLUDE where productionDate is  null""",
-      s"""select productionDate from TABLE_DICTIONARY_EXCLUDE1_hive where productionDate is  null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_385")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_386
-  test("Queries_DICTIONARY_EXCLUDE_386", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE where deviceInformationId is  null""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_EXCLUDE1_hive where deviceInformationId is  null""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_386")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_387
-  test("Queries_DICTIONARY_EXCLUDE_387", Include) {
-
-    checkAnswer(s"""select count(*) from TABLE_DICTIONARY_EXCLUDE where imei = '1AA1'""",
-      s"""select count(*) from TABLE_DICTIONARY_EXCLUDE1_hive where imei = '1AA1'""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_387")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_001
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_001", Include) {
-
-    checkAnswer(s"""select count(imei)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000'  and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(imei)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000'  and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_001")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_002
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_002", Include) {
-
-    checkAnswer(s"""select count(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_002")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_003
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_003", Include) {
-
-    checkAnswer(s"""select count(productionDate)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(productionDate)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_003")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_004
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_004", Include) {
-
-    checkAnswer(s"""select count(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(gamePointId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_004")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_005
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_005", Include) {
-
-    checkAnswer(s"""select count(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_005")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_006
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_006", Include) {
-
-    checkAnswer(s"""select count(contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_006")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_007
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_007", Include) {
-
-    checkAnswer(s"""select count(distinct imei)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct imei)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_007")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_008
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_008", Include) {
-
-    checkAnswer(s"""select count(distinct deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_008")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_009
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_009", Include) {
-
-    checkAnswer(s"""select count(distinct productionDate)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct productionDate)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_009")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_010
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_010", Include) {
-
-    checkAnswer(s"""select count(distinct gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct gamePointId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_010")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_011
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_011", Include) {
-
-    checkAnswer(s"""select count(distinct Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_011")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_012
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_012", Include) {
-
-    checkAnswer(s"""select count(distinct contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_012")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_013
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_013", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_013")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_014
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_014", Include) {
-
-    checkAnswer(s"""select sum(productionDate)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(productionDate)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_014")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_015
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_015", Include) {
-
-    sql(s"""select sum(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_016
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_016", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_016")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_017
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_017", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_017")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_018
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_018", Include) {
-
-    checkAnswer(s"""select sum(distinct deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_018")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_019
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_019", Include) {
-
-    checkAnswer(s"""select sum(distinct gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct gamePointId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_019")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_020
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_020", Include) {
-
-    checkAnswer(s"""select sum(distinct Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_020")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_021
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_021", Include) {
-
-    checkAnswer(s"""select sum(distinct contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_021")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_022
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_022", Include) {
-
-    checkAnswer(s"""select min(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_022")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_023
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_023", Include) {
-
-    checkAnswer(s"""select min(productionDate)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(productionDate)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_023")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_024
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_024", Include) {
-
-    checkAnswer(s"""select min(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(gamePointId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_024")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_025
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_025", Include) {
-
-    checkAnswer(s"""select min(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_025")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_026
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_026", Include) {
-
-    checkAnswer(s"""select min(contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_026")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_027
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_027", Include) {
-
-    checkAnswer(s"""select max(imei)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(imei)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_027")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_028
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_028", Include) {
-
-    checkAnswer(s"""select max(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_028")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_029
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_029", Include) {
-
-    checkAnswer(s"""select max(productionDate)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(productionDate)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_029")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_030
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_030", Include) {
-
-    checkAnswer(s"""select max(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(gamePointId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_030")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_031
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_031", Include) {
-
-    checkAnswer(s"""select max(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_031")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_032
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_032", Include) {
-
-    checkAnswer(s"""select max(contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_032")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_033
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_033", Include) {
-
-    checkAnswer(s"""select variance(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select variance(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_033")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_034
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_034", Include) {
-
-    checkAnswer(s"""select variance(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select variance(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_034")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_035
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_035", Include) {
-
-    sql(s"""select variance(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_036
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_036", Include) {
-
-    checkAnswer(s"""select variance(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select variance(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_036")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_037
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_037", Include) {
-
-    checkAnswer(s"""select variance(contractNumber) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by contractNumber)""",
-      s"""select variance(contractNumber) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by contractNumber)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_037")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_038
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_038", Include) {
-
-    checkAnswer(s"""select var_samp(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select var_samp(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_038")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_039
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_039", Include) {
-
-    checkAnswer(s"""select var_samp(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select var_samp(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_039")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_040
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_040", Include) {
-
-    sql(s"""select var_samp(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_041
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_041", Include) {
-
-    checkAnswer(s"""select var_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select var_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_041")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_042
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_042", Include) {
-
-    sql(s"""select var_samp(contractNumber) from (select contractNumber from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')order by contractNumber)""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_043
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_043", Include) {
-
-    checkAnswer(s"""select stddev_pop(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select stddev_pop(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_043")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_044
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_044", Include) {
-
-    checkAnswer(s"""select stddev_pop(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select stddev_pop(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_044")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_045
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_045", Include) {
-
-    sql(s"""select stddev_pop(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_046
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_046", Include) {
-
-    checkAnswer(s"""select stddev_pop(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select stddev_pop(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_046")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_047
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_047", Include) {
-
-    checkAnswer(s"""select stddev_pop(contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select stddev_pop(contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_047")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_048
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_048", Include) {
-
-    checkAnswer(s"""select stddev_samp(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select stddev_samp(deviceInformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_048")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_049
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_049", Include) {
-
-    checkAnswer(s"""select stddev_samp(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select stddev_samp(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_049")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_050
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_050", Include) {
-
-    checkAnswer(s"""select stddev_samp(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select stddev_samp(gamePointId) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_050")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_051
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_051", Include) {
-
-    checkAnswer(s"""select stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_051")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_053
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_053", Include) {
-
-    checkAnswer(s"""select count(imei),count(distinct deviceinformationId),sum(deviceinformationId),sum(distinct deviceinformationId),min(deviceinformationId),max(imei),variance(deviceInformationId),var_samp(deviceInformationId),stddev_pop(gamePointId),stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by deviceinformationId)""",
-      s"""select count(imei),count(distinct deviceinformationId),sum(deviceinformationId),sum(distinct deviceinformationId),min(deviceinformationId),max(imei),variance(deviceInformationId),var_samp(deviceInformationId),stddev_pop(gamePointId),stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by deviceinformationId)""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_053")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_054
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_054", Include) {
-
-    checkAnswer(s"""select AVG(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(deviceinformationId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_054")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_055
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_055", Include) {
-
-    checkAnswer(s"""select AVG(productionDate)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(productionDate)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_055")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_056
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_056", Include) {
-
-    checkAnswer(s"""select AVG(gamePointId)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(gamePointId)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_056")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_057
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_057", Include) {
-
-    checkAnswer(s"""select AVG(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(Latest_DAY)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_057")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_058
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_058", Include) {
-
-    checkAnswer(s"""select AVG(contractNumber)  from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(contractNumber)  from TABLE_DICTIONARY_EXCLUDE1_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_058")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_059
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_059", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE group by deviceInformationId limit 5""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_060
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_060", Include) {
-
-    sql(s"""select sum(deviceinformationId),sum(distinct deviceinformationId),min(deviceinformationId),max(imei),variance(deviceInformationId) from TABLE_DICTIONARY_EXCLUDE group by deviceInformationId order by deviceinformationId limit 5""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_061
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_061", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')group by deviceInformationId limit 5""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_062
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_062", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') group by deviceInformationId order by deviceinformationId limit 5""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_063
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_063", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_EXCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') group by deviceInformationId,productionDate   sort by productionDate limit 5
-  """).collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_064
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_064", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId+10)t  from  TABLE_DICTIONARY_EXCLUDE having t >1234567""",
-      s"""select sum(deviceinformationId+10)t  from  TABLE_DICTIONARY_EXCLUDE1_hive having t >1234567""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_064")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_065
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_065", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId+gamePointId)t  from  (select gamePointId,deviceinformationId from TABLE_DICTIONARY_EXCLUDE order by gamePointId,deviceinformationId) having t >1234567""",
-      s"""select sum(deviceinformationId+gamePointId)t  from  (select gamePointId,deviceinformationId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId,deviceinformationId) having t >1234567""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_065")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_066
-  ignore("Queries_DICTIONARY_EXCLUDE_PushUP_066", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId)t,Sum(gamePointId)   from  (select gamePointId,deviceinformationId from TABLE_DICTIONARY_EXCLUDE order by gamePointId,deviceinformationId) having t >1234567""",
-      s"""select sum(deviceinformationId)t,Sum(gamePointId)   from  (select gamePointId,deviceinformationId from TABLE_DICTIONARY_EXCLUDE1_hive order by gamePointId,deviceinformationId) having t >1234567""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_066")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_067
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_067", Include) {
-
-    checkAnswer(s"""select count(imei) a,sum(distinct deviceinformationId) b,min(productionDate) c  from TABLE_DICTIONARY_EXCLUDE group by imei,deviceinformationId,productionDate  order by  a,b,c""",
-      s"""select count(imei) a,sum(distinct deviceinformationId) b,min(productionDate) c  from TABLE_DICTIONARY_EXCLUDE1_hive group by imei,deviceinformationId,productionDate  order by  a,b,c""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_067")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_069
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_069", Include) {
-
-    sql(s"""SELECT  min(Latest_DAY),max(imei),variance(contractNumber), SUM(gamePointId),count(imei),sum(distinct deviceinformationId) FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY GROUP BY AMSize, ActiveAreaId ORDER BY AMSize ASC, ActiveAreaId ASC limit 10""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_070
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_070", Include) {
-
-    checkAnswer(s"""SELECT  TABLE_DICTIONARY_EXCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_EXCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE FULL OUTER JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE1 ON TABLE_DICTIONARY_EXCLUDE.AMSize = TABLE_DICTIONARY_EXCLUDE1.AMSize WHERE TABLE_DICTIONARY_EXCLUDE.AMSize LIKE '5RAM %' GROUP BY TABLE_DICTIONARY_EXCLUDE.AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity,TABLE_DICTIONARY_EXCLUDE.gamePointId ORDER BY TABLE_DICTIONARY_EXCLUDE.gamePointId ASC, TABLE_DICTIONARY_EXCLUDE.AMSize ASC, TABLE_DICTIONARY_EXCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_EXCLUDE.Activecity ASC""",
-      s"""SELECT  TABLE_DICTIONARY_EXCLUDE1_hive.gamePointId AS gamePointId,TABLE_DICTIONARY_EXCLUDE1_hive.AMSize AS AMSize, TABLE_DICTIONARY_EXCLUDE1_hive.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_EXCLUDE1_hive.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_EXCLUDE1_hive) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE1_hive FULL OUTER JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_EXCLUDE1_hive) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE1_hive1 ON TABLE_DICTIONARY_EXCLUDE1_hive.AMSize = TABLE_DICTIONARY_EXCLUDE1_hive1.AMSize WHERE TABLE_DICTIONARY_EXCLUDE1_hive.AMSize LIKE '5RAM %' GROUP BY TABLE_DICTIONARY_EXCLUDE1_hive.AMSize, TABLE_DICTIONARY_EXCLUDE1_hive.ActiveCountry, TABLE_DICTIONARY_EXCLUDE1_hive.Activecity,TABLE_DICTIONARY_EXCLUDE1_hive.gamePointId ORDER BY TABLE_DICTIONARY_EXCLUDE1_hive.gamePointId ASC, TABLE_DICTIONARY_EXCLUDE1_hive.AMSize ASC, TABLE_DICTIONARY_EXCLUDE1_hive.ActiveCountry ASC, TABLE_DICTIONARY_EXCLUDE1_hive.Activecity ASC""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_070")
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_071
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_071", Include) {
-
-    sql(s"""SELECT TABLE_DICTIONARY_EXCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_EXCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE RIGHT JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE1 ON TABLE_DICTIONARY_EXCLUDE.AMSize = TABLE_DICTIONARY_EXCLUDE1.AMSize WHERE NOT(TABLE_DICTIONARY_EXCLUDE.AMSize = "8RAM size") GROUP BY TABLE_DICTIONARY_EXCLUDE.AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity,TABLE_DICTIONARY_EXCLUDE.gamePointId  ORDER BY TABLE_DICTIONARY_EXCLUDE.AMSize ASC, TABLE_DICTIONARY_EXCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_EXCLUDE.Activecity ASC limit 10""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_072
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_072", Include) {
-
-    sql(s"""SELECT TABLE_DICTIONARY_EXCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_EXCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE LEFT JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE1 ON TABLE_DICTIONARY_EXCLUDE.AMSize = TABLE_DICTIONARY_EXCLUDE1.AMSize WHERE NOT(TABLE_DICTIONARY_EXCLUDE.AMSize = "8RAM size") GROUP BY TABLE_DICTIONARY_EXCLUDE.AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity,TABLE_DICTIONARY_EXCLUDE.gamePointId  ORDER BY TABLE_DICTIONARY_EXCLUDE.AMSize ASC, TABLE_DICTIONARY_EXCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_EXCLUDE.Activecity ASC limit 10
-  """).collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_073
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_073", Include) {
-
-    sql(s"""SELECT TABLE_DICTIONARY_EXCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_EXCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE INNER JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_EXCLUDE) SUB_QRY ) TABLE_DICTIONARY_EXCLUDE1 ON TABLE_DICTIONARY_EXCLUDE.AMSize = TABLE_DICTIONARY_EXCLUDE1.AMSize WHERE NOT(TABLE_DICTIONARY_EXCLUDE.AMSize = "8RAM size") GROUP BY TABLE_DICTIONARY_EXCLUDE.AMSize, TABLE_DICTIONARY_EXCLUDE.ActiveCountry, TABLE_DICTIONARY_EXCLUDE.Activecity,TABLE_DICTIONARY_EXCLUDE.gamePointId  ORDER BY TABLE_DICTIONARY_EXCLUDE.AMSize ASC, TABLE_DICTIONARY_EXCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_EXCLUDE.Activecity ASC limit 10
-  """).collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_074
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_074", Include) {
-
-    sql(s"""select count(gamepointid),series  from TABLE_DICTIONARY_EXCLUDE group by series order by series limit 5""").collect
-
-  }
-
-
-  //DICTIONARY_EXCLUDE_PushUP_075
-  test("Queries_DICTIONARY_EXCLUDE_PushUP_075", Include) {
-
-    checkAnswer(s"""select count(gamepointid),series  from TABLE_DICTIONARY_EXCLUDE group by series order by series""",
-      s"""select count(gamepointid),series  from TABLE_DICTIONARY_EXCLUDE1_hive group by series order by series""", "QueriesExcludeDictionaryTestCase_DICTIONARY_EXCLUDE_PushUP_075")
-
-  }
-
-
-  //C20_SEQ_CreateTable-DICTIONARY_EXCLUDE-01
-  test("C20_SEQ_CreateTable-DICTIONARY_EXCLUDE-01", Include) {
-
-    sql(s"""create table smart_500_DE (MSISDN string,IMSI string,IMEI string,INTERFACEID int,GROUPID int,GGSN_ID double,SGSN_ID double,SESSION_INDICATOR int,BEGIN_TIME double,BEGIN_TIME_MSEL int,END_TIME double,END_TIME_MSEL int,PROT_CATEGORY int,PROT_TYPE int,L7_CARRIER_PROT int,SUB_PROT_TYPE int,SID double,PROBEID double,ENCRYPT_VERSION int,ROAMING_TYPE int,ROAM_DIRECTION int,MS_IP string,SERVER_IP string,MS_PORT int,SERVER_DECIMAL Decimal,APN string,SGSN_SIG_IP string,GGSN_SIG_IP_BigInt_NEGATIVE bigint,SGSN_USER_IP string,GGSN_USER_IP string,MCC string,MNC string,RAT int,LAC string,RAC string,SAC string,CI string,BROWSER_TIMESTAMP timestamp,TCP_CONN_STATES int,TCP_STATES_BIGINTPOSITIVE int,TCP_WIN_SIZE int,TCP_MSS int,TCP_CONN_TIMES int,TCP_CONN_2_FAILED_TIMES int,TCP_CONN_3_FAILED_TIMES int,HOST string,STREAMING_URL string,GET_STREAMING_FAILED_CODE int,GET_STREAMING_FLAG int,GET_NUM int,GET_SUCCEED_NUM int,GET_RETRANS_NUM int,GET_TIMEOUT_NUM int,INTBUFFER_FST_FLAG int,INTBUFFER_FULL_FLAG int,STALL_NUM int,VIDEO_FRAME_RATE int,VIDEO_CODEC_ID string,VIDEO_WIDTH int,VIDEO_HEIGHT int,AUDIO_CODEC_ID string,MEDIA_FILE_TYPE int,PLAY_STATE int,STREAMING_FLAG int,TCP_STATUS_INDICATOR int,DISCONNECTION_FLAG int,FAILURE_CODE int,FLAG int,TAC string,ECI string,TCP_SYN_TIME_MSEL int,TCP_FST_SYN_DIRECTION int,RAN_NE_USER_IP string,HOMEMCC string,HOMEMNC string,CHARGE_FLAG int,PREPAID_FLAG int,USER_AGENT string,MS_WIN_STAT_TOTAL_NUM int,MS_WIN_STAT_SMALL_NUM int,MS_ACK_TO_1STGET_DELAY int,SERVER_ACK_TO_1STDATA_DELAY int,STREAMING_TYPE int,SOURCE_VIDEO_QUALITY int,TETHERING_FLAG int,CARRIER_ID double,LAYER1ID int,LAYER2ID int,LAYER3ID int,LAYER4ID int,LAYER5ID int,LAYER6ID int,CHARGING_RULE_BASE_NAME string,SP string,EXTENDED_URL string,SV string,FIRST_SAI_CGI_ECGI string,EXTENDED_URL_OTHER string,SIGNALING_USE_FLAG int,DNS_RETRANS_NUM int,DNS_FAIL_CODE int,FIRST_RAT int,MS_INDICATOR string,LAST_SAI_CGI_ECGI string,LAST_RAT int,FIRST_LONGITUDE double,FIRST_LATITUDE double,FIRST_ALTITUDE int,FIRST_RASTERLONGITUDE double,FIRST_RASTERLATITUDE double,FIRST_RASTERALTITUDE int,FIRST_FREQUENCYSPOT int,FIRST_CLUTTER int,FIRST_USERBEHAVIOR int,FIRST_SPEED int,FIRST_CREDIBILITY int,LAST_LONGITUDE double,LAST_LATITUDE double,LAST_ALTITUDE int,LAST_RASTERLONGITUDE double,LAST_RASTERLATITUDE double,LAST_RASTERALTITUDE int,LAST_FREQUENCYSPOT int,LAST_CLUTTER int,LAST_USERBEHAVIOR int,LAST_SPEED int,LAST_CREDIBILITY int,IMEI_CIPHERTEXT string,APP_ID int,DOMAIN_NAME string,STREAMING_CACHE_IP string,STOP_LONGER_THAN_MIN_THRESHOLD int,STOP_LONGER_THAN_MAX_THRESHOLD int,PLAY_END_STAT int,STOP_START_TIME1 double,STOP_END_TIME1 double,STOP_START_TIME2 double,STOP_END_TIME2 double,STOP_START_TIME3 double,STOP_END_TIME3 double,STOP_START_TIME4 double,STOP_END_TIME4 double,STOP_START_TIME5 double,STOP_END_TIME5 double,STOP_START_TIME6 double,STOP_END_TIME6 double,STOP_START_TIME7 double,STOP_END_TIME7 double,STOP_START_TIME8 double,STOP_END_TIME8 double,STOP_START_TIME9 double,STOP_END_TIME9 double,STOP_START_TIME10 double,STOP_END_TIME10 double,FAIL_CLASS double,RECORD_TYPE double,NODATA_COUNT double,VIDEO_NODATA_DURATION double,VIDEO_SMOOTH_DURATION double,VIDEO_SD_DURATION double,VIDEO_HD_DURATION double,VIDEO_UHD_DURATION double,VIDEO_FHD_DURATION double,FLUCTUATION double,START_DOWNLOAD_THROUGHPUT double,L7_UL_GOODPUT_FULL_MSS double,SESSIONKEY string,FIRST_UCELLID double,LAST_UCELLID double,UCELLID1 double,LONGITUDE1 double,LATITUDE1 double,UCELLID2 double,LONGITUDE2 double,LATITUDE2 double,UCELLID3 double,LONGITUDE3 double,LATITUDE3 double,UCELLID4 double,LONGITUDE4 double,LATITUDE4 double,UCELLID5 double,LONGITUDE5 double,LATITUDE5 double,UCELLID6 double,LONGITUDE6 double,LATITUDE6 double,UCELLID7 double,LONGITUDE7 double,LATITUDE7 double,UCELLID8 double,LONGITUDE8 double,LATITUDE8 double,UCELLID9 double,LONGITUDE9 double,LATITUDE9 double,UCELLID10 double,LONGITUDE10 double,LATITUDE10 double,INTBUFFER_FULL_DELAY double,STALL_DURATION double,STREAMING_DW_PACKETS double,STREAMING_DOWNLOAD_DELAY double,PLAY_DURATION double,STREAMING_QUALITY int,VIDEO_DATA_RATE double,AUDIO_DATA_RATE double,STREAMING_FILESIZE double,STREAMING_DURATIOIN double,TCP_SYN_TIME double,TCP_RTT_STEP1 double,CHARGE_ID double,UL_REVERSE_TO_DL_DELAY double,DL_REVERSE_TO_UL_DELAY double,DATATRANS_DW_GOODPUT double,DATATRANS_DW_TOTAL_DURATION double,SUM_FRAGMENT_INTERVAL double,TCP_FIN_TIMES double,TCP_RESET_TIMES double,URL_CLASSIFICATION double,STREAMING_LQ_DURATIOIN double,MAX_DNS_DELAY double,MAX_DNS2SYN double,MAX_LATANCY_OF_LINK_SETUP double,MAX_SYNACK2FIRSTACK double,MAX_SYNACK2LASTACK double,MAX_ACK2GET_DELAY double,MAX_FRAG_INTERVAL_PREDELAY double,SUM_FRAG_INTERVAL_PREDELAY double,SERVICE_DELAY_MSEC double,HOMEPROVINCE double,HOMECITY double,SERVICE_ID double,CHARGING_CLASS double,DATATRANS_UL_DURATION double,ASSOCIATED_ID double,PACKET_LOSS_NUM double,JITTER double,MS_DNS_DELAY_MSEL double,GET_STREAMING_DELAY double,TCP_UL_RETRANS_WITHOUTPL double,TCP_DW_RETRANS_WITHOUTPL double,GET_MAX_UL_SIZE double,GET_MIN_UL_SIZE double,GET_MAX_DL_SIZE double,GET_MIN_DL_SIZE double,L4_UL_THROUGHPUT double,L4_DW_THROUGHPUT double,L4_UL_GOODPUT double,L4_DW_GOODPUT double,NETWORK_UL_TRAFFIC double,NETWORK_DL_TRAFFIC double,L4_UL_PACKETS double,L4_DW_PACKETS double,TCP_RTT double,TCP_UL_OUTOFSEQU double,TCP_DW_OUTOFSEQU double,TCP_UL_RETRANS double,TCP_DW_RETRANS double,TCP_UL_RETRANS_WITHPL double,TCP_DW_RETRANS_WITHPL double,TCP_UL_PACKAGES_WITHPL double,TCP_DW_PACKAGES_WITHPL double,TCP_UL_PACKAGES_WITHOUTPL double,TCP_DW_PACKAGES_WITHOUTPL double,UPPERLAYER_IP_UL_PACKETS double,UPPERLAYER_IP_DL_PACKETS double,DOWNLAYER_IP_UL_PACKETS double,DOWNLAYER_IP_DL_PACKETS double,UPPERLAYER_IP_UL_FRAGMENTS double,UPPERLAYER_IP_DL_FRAGMENTS double,DOWNLAYER_IP_UL_FRAGMENTS double,DOWNLAYER_IP_DL_FRAGMENTS double,VALID_TRANS_DURATION double,AIR_PORT_DURATION double,RADIO_CONN_TIMES double,RAN_NE_ID double,AVG_UL_RTT double,AVG_DW_RTT double,UL_RTT_LONG_NUM int,DW_RTT_LONG_NUM int,UL_RTT_STAT_NUM int,DW_RTT_STAT_NUM int,USER_PROBE_UL_LOST_PKT int,SERVER_PROBE_UL_LOST_PKT int,SERVER_PROBE_DW_LOST_PKT int,USER_PROBE_DW_LOST_PKT int,CHARGING_CHARACTERISTICS double,DL_SERIOUS_OUT_OF_ORDER_NUM double,DL_SLIGHT_OUT_OF_ORDER_NUM double,DL_FLIGHT_TOTAL_SIZE double,DL_FLIGHT_TOTAL_NUM double,DL_MAX_FLIGHT_SIZE double,UL_SERIOUS_OUT_OF_ORDER_NUM double,UL_SLIGHT_OUT_OF_ORDER_NUM double,UL_FLIGHT_TOTAL_SIZE double,UL_FLIGHT_TOTAL_NUM double,UL_MAX_FLIGHT_SIZE double,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS double,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS double,DL_CONTINUOUS_RETRANSMISSION_DELAY double,USER_HUNGRY_DELAY double,SERVER_HUNGRY_DELAY double,AVG_DW_RTT_MICRO_SEC int,AVG_UL_RTT_MICRO_SEC int,FLOW_SAMPLE_RATIO int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,IMSI,IMEI,MS_IP,SERVER_IP,HOST,SP,MS_INDICATOR,streaming_url','DICTIONARY_INCLUDE'='SESSION_INDICATOR,SERVER_DECIMAL,TCP_STATES_BIGINTPOSITIVE')""").collect
-
-    sql(s"""create table smart_500_DE_hive (SID double,PROBEID double,INTERFACEID int,GROUPID int,GGSN_ID double,SGSN_ID double,dummy_6 string,SESSION_INDICATOR int,BEGIN_TIME double,BEGIN_TIME_MSEL int,END_TIME double,END_TIME_MSEL int,PROT_CATEGORY int,PROT_TYPE int,L7_CARRIER_PROT int,SUB_PROT_TYPE int,MSISDN string,IMSI string,IMEI string,ENCRYPT_VERSION int,ROAMING_TYPE int,ROAM_DIRECTION int,MS_IP string,SERVER_IP string,MS_PORT int,APN string,SGSN_SIG_IP string,GGSN_USER_IP string,SGSN_USER_IP string,MCC string,MNC string,RAT int,LAC string,RAC string,SAC string,CI string,SERVER_DECIMAL decimal,BROWSER_TIMESTAMP timestamp,TCP_CONN_STATES int,GGSN_SIG_IP_BigInt_NEGATIVE bigint,TCP_STATES_BIGINTPOSITIVE bigint,dummy_41 string,TCP_WIN_SIZE int,dummy_43 string,TCP_MSS int,dummy_45 string,TCP_CONN_TIMES int,dummy_47 string,TCP_CONN_2_FAILED_TIMES int,dummy_49 string,TCP_CONN_3_FAILED_TIMES int,HOST string,STREAMING_URL string,dummy_53 string,GET_STREAMING_FAILED_CODE int,dummy_55 string,GET_STREAMING_FLAG int,dummy_57 string,GET_NUM int,dummy_59 string,GET_SUCCEED_NUM int,dummy_61 string,GET_RETRANS_NUM int,dummy_63 string,GET_TIMEOUT_NUM int,INTBUFFER_FST_FLAG int,INTBUFFER_FULL_FLAG int,STALL_NUM int,dummy_68 string,VIDEO_FRAME_RATE int,dummy_70 string,VIDEO_CODEC_ID string,dummy_72 string,VIDEO_WIDTH int,dummy_74 string,VIDEO_HEIGHT int,dummy_76 string,AUDIO_CODEC_ID string,dummy_78 string,MEDIA_FILE_TYPE int,dummy_80 string,PLAY_STATE int,dummy_82 string,PLAY_STATE_1 int,dummy_84 string,STREAMING_FLAG int,dummy_86 string,TCP_STATUS_INDICATOR int,dummy_88 string,DISCONNECTION_FLAG int,dummy_90 string,FAILURE_CODE int,FLAG int,TAC string,ECI string,dummy_95 string,TCP_SYN_TIME_MSEL int,dummy_97 string,TCP_FST_SYN_DIRECTION int,RAN_NE_USER_IP string,HOMEMCC string,HOMEMNC string,dummy_102 string,CHARGE_FLAG int,dummy_104 string,PREPAID_FLAG int,dummy_106 string,USER_AGENT string,dummy_108 string,MS_WIN_STAT_TOTAL_NUM int,dummy_110 string,MS_WIN_STAT_SMALL_NUM int,dummy_112 string,MS_ACK_TO_1STGET_DELAY int,dummy_114 string,SERVER_ACK_TO_1STDATA_DELAY int,dummy_116 string,STREAMING_TYPE int,dummy_118 string,SOURCE_VIDEO_QUALITY int,TETHERING_FLAG int,CARRIER_ID double,LAYER1ID int,LAYER2ID int,dummy_124 string,LAYER3ID int,dummy_126 string,LAYER4ID int,dummy_128 string,LAYER5ID int,dummy_130 string,LAYER6ID int,CHARGING_RULE_BASE_NAME string,SP string,dummy_134 string,EXTENDED_URL string,SV string,FIRST_SAI_CGI_ECGI string,dummy_138 string,EXTENDED_URL_OTHER string,SIGNALING_USE_FLAG int,dummy_141 string,DNS_RETRANS_NUM int,dummy_143 string,DNS_FAIL_CODE int,FIRST_RAT int,FIRST_RAT_1 int,MS_INDICATOR string,LAST_SAI_CGI_ECGI string,LAST_RAT int,dummy_150 string,FIRST_LONGITUDE double,dummy_152 string,FIRST_LATITUDE double,dummy_154 string,FIRST_ALTITUDE int,dummy_156 string,FIRST_RASTERLONGITUDE double,dummy_158 string,FIRST_RASTERLATITUDE double,dummy_160 string,FIRST_RASTERALTITUDE int,dummy_162 string,FIRST_FREQUENCYSPOT int,dummy_164 string,FIRST_CLUTTER int,dummy_166 string,FIRST_USERBEHAVIOR int,dummy_168 string,FIRST_SPEED int,dummy_170 string,FIRST_CREDIBILITY int,dummy_172 string,LAST_LONGITUDE double,dummy_174 string,LAST_LATITUDE double,dummy_176 string,LAST_ALTITUDE int,dummy_178 string,LAST_RASTERLONGITUDE double,dummy_180 string,LAST_RASTERLATITUDE double,dummy_182 string,LAST_RASTERALTITUDE int,dummy_184 string,LAST_FREQUENCYSPOT int,dummy_186 string,LAST_CLUTTER int,dummy_188 string,LAST_USERBEHAVIOR int,dummy_190 string,LAST_SPEED int,dummy_192 string,LAST_CREDIBILITY int,dummy_194 string,IMEI_CIPHERTEXT string,APP_ID int,dummy_197 string,DOMAIN_NAME string,dummy_199 string,STREAMING_CACHE_IP string,dummy_201 string,STOP_LONGER_THAN_MIN_THRESHOLD int,dummy_203 string,STOP_LONGER_THAN_MAX_THRESHOLD int,dummy_205 string,PLAY_END_STAT int,dummy_207 string,STOP_START_TIME1 double,dummy_209 string,STOP_END_TIME1 double,dummy_211 string,STOP_START_TIME2 double,dummy_213 string,STOP_END_TIME2 double,dummy_215 string,STOP_START_TIME3 double,dummy_217 string,STOP_END_TIME3 double,dummy_219 string,STOP_START_TIME4 double,dummy_221 string,STOP_END_TIME4 double,dummy_223 string,STOP_START_TIME5 double,dummy_225 string,STOP_END_TIME5 double,dummy_227 string,STOP_START_TIME6 double,dummy_229 string,STOP_END_TIME6 double,dummy_231 string,STOP_START_TIME7 double,dummy_233 string,STOP_END_TIME7 double,dummy_235 string,STOP_START_TIME8 double,dummy_237 string,STOP_END_TIME8 double,dummy_239 string,STOP_START_TIME9 double,dummy_241 string,STOP_END_TIME9 double,dummy_243 string,STOP_START_TIME10 double,dummy_245 string,STOP_END_TIME10 double,dummy_247 string,FAIL_CLASS double,RECORD_TYPE double,dummy_250 string,NODATA_COUNT double,dummy_252 string,VIDEO_NODATA_DURATION double,dummy_254 string,VIDEO_SMOOTH_DURATION double,dummy_256 string,VIDEO_SD_DURATION double,dummy_258 string,VIDEO_HD_DURATION double,dummy_260 string,VIDEO_UHD_DURATION double,dummy_262 string,VIDEO_FHD_DURATION double,dummy_264 string,FLUCTUATION double,dummy_266 string,START_DOWNLOAD_THROUGHPUT double,dummy_268 string,L7_UL_GOODPUT_FULL_MSS double,dummy_270 string,SESSIONKEY string,dummy_272 string,FIRST_UCELLID double,dummy_274 string,LAST_UCELLID double,dummy_276 string,UCELLID1 double,dummy_278 string,LONGITUDE1 double,dummy_280 string,LATITUDE1 double,dummy_282 string,UCELLID2 double,dummy_284 string,LONGITUDE2 double,dummy_286 string,LATITUDE2 double,dummy_288 string,UCELLID3 double,dummy_290 string,LONGITUDE3 double,dummy_292 string,LATITUDE3 double,dummy_294 string,UCELLID4 double,dummy_296 string,LONGITUDE4 double,dummy_2101 string,LATITUDE4 double,dummy_300 string,UCELLID5 double,dummy_302 string,LONGITUDE5 double,dummy_304 string,LATITUDE5 double,dummy_306 string,UCELLID6 double,dummy_308 string,LONGITUDE6 double,dummy_310 string,LATITUDE6 double,dummy_312 string,UCELLID7 double,dummy_314 string,LONGITUDE7 double,dummy_316 string,LATITUDE7 double,dummy_318 string,UCELLID8 double,dummy_320 string,LONGITUDE8 double,dummy_322 string,LATITUDE8 double,dummy_324 string,UCELLID9 double,dummy_326 string,LONGITUDE9 double,dummy_328 string,LATITUDE9 double,dummy_330 string,UCELLID10 double,dummy_332 string,LONGITUDE10 double,dummy_334 string,LATITUDE10 double,dummy_336 string,INTBUFFER_FULL_DELAY double,dummy_338 string,STALL_DURATION double,dummy_340 string,STREAMING_DW_PACKETS double,dummy_342 string,STREAMING_DOWNLOAD_DELAY double,dummy_344 string,PLAY_DURATION double,dummy_346 string,STREAMING_QUALITY int,dummy_348 string,VIDEO_DATA_RATE double,dummy_350 string,AUDIO_DATA_RATE double,dummy_352 string,STREAMING_FILESIZE double,dummy_354 string,STREAMING_DURATIOIN double,dummy_356 string,TCP_SYN_TIME double,dummy_358 string,TCP_RTT_STEP1 double,CHARGE_ID double,dummy_361 string,UL_REVERSE_TO_DL_DELAY double,dummy_363 string,DL_REVERSE_TO_UL_DELAY double,dummy_365 string,DATATRANS_DW_GOODPUT double,dummy_367 string,DATATRANS_DW_TOTAL_DURATION double,dummy_369 string,SUM_FRAGMENT_INTERVAL double,dummy_371 string,TCP_FIN_TIMES double,dummy_373 string,TCP_RESET_TIMES double,dummy_375 string,URL_CLASSIFICATION double,dummy_377 string,STREAMING_LQ_DURATIOIN double,dummy_379 string,MAX_DNS_DELAY double,dummy_381 string,MAX_DNS2SYN double,dummy_383 string,MAX_LATANCY_OF_LINK_SETUP double,dummy_385 string,MAX_SYNACK2FIRSTACK double,dummy_387 string,MAX_SYNACK2LASTACK double,dummy_389 string,MAX_ACK2GET_DELAY double,dummy_391 string,MAX_FRAG_INTERVAL_PREDELAY double,dummy_393 string,SUM_FRAG_INTERVAL_PREDELAY double,dummy_395 string,SERVICE_DELAY_MSEC double,dummy_397 string,HOMEPROVINCE double,dummy_399 string,HOMECITY double,dummy_401 string,SERVICE_ID double,dummy_403 string,CHARGING_CLASS double,dummy_405 string,DATATRANS_UL_DURATION double,dummy_407 string,ASSOCIATED_ID double,dummy_409 string,PACKET_LOSS_NUM double,dummy_411 string,JITTER double,dummy_413 string,MS_DNS_DELAY_MSEL double,dummy_415 string,GET_STREAMING_DELAY double,dummy_417 string,TCP_UL_RETRANS_WITHOUTPL double,dummy_419 string,TCP_DW_RETRANS_WITHOUTPL double,dummy_421 string,GET_MAX_UL_SIZE double,dummy_423 string,GET_MIN_UL_SIZE double,dummy_425 string,GET_MAX_DL_SIZE double,dummy_427 string,GET_MIN_DL_SIZE double,dummy_429 string,FLOW_SAMPLE_RATIO int,dummy_431 string,UL_RTT_LONG_NUM int,dummy_433 string,DW_RTT_LONG_NUM int,dummy_435 string,UL_RTT_STAT_NUM int,dummy_437 string,DW_RTT_STAT_NUM int,dummy_439 string,USER_PROBE_UL_LOST_PKT int,dummy_441 string,SERVER_PROBE_UL_LOST_PKT int,dummy_443 string,SERVER_PROBE_DW_LOST_PKT int,dummy_445 string,USER_PROBE_DW_LOST_PKT int,dummy_447 string,AVG_DW_RTT_MICRO_SEC int,dummy_449 string,AVG_UL_RTT_MICRO_SEC int,dummy_451 string,RAN_NE_ID double,dummy_453 string,AVG_UL_RTT double,dummy_455 string,AVG_DW_RTT double,dummy_457 string,CHARGING_CHARACTERISTICS double,dummy_459 string,DL_SERIOUS_OUT_OF_ORDER_NUM double,dummy_461 string,DL_SLIGHT_OUT_OF_ORDER_NUM double,dummy_463 string,DL_FLIGHT_TOTAL_SIZE double,dummy_465 string,DL_FLIGHT_TOTAL_NUM double,dummy_467 string,DL_MAX_FLIGHT_SIZE double,dummy_469 string,VALID_TRANS_DURATION double,dummy_471 string,AIR_PORT_DURATION double,dummy_473 string,RADIO_CONN_TIMES double,dummy_475 string,UL_SERIOUS_OUT_OF_ORDER_NUM double,dummy_477 string,UL_SLIGHT_OUT_OF_ORDER_NUM double,dummy_479 string,UL_FLIGHT_TOTAL_SIZE double,dummy_481 string,UL_FLIGHT_TOTAL_NUM double,dummy_483 string,UL_MAX_FLIGHT_SIZE double,dummy_485 string,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS double,dummy_487 string,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS double,dummy_489 string,DL_CONTINUOUS_RETRANSMISSION_DELAY double,dummy_491 string,USER_HUNGRY_DELAY double,dummy_493 string,SERVER_HUNGRY_DELAY double,dummy_495 string,UPPERLAYER_IP_UL_FRAGMENTS double,dummy_497 string,UPPERLAYER_IP_DL_FRAGMENTS double,dummy_499 string,DOWNLAYER_IP_UL_FRAGMENTS double,dummy_501 string,DOWNLAYER_IP_DL_FRAGMENTS double,dummy_503 string,UPPERLAYER_IP_UL_PACKETS double,dummy_505 string,UPPERLAYER_IP_DL_PACKETS double,dummy_507 string,DOWNLAYER_IP_UL_PACKETS double,dummy_509 string,DOWNLAYER_IP_DL_PACKETS double,dummy_511 string,TCP_UL_PACKAGES_WITHPL double,dummy_513 string,TCP_DW_PACKAGES_WITHPL double,dummy_515 string,TCP_UL_PACKAGES_WITHOUTPL double,dummy_517 string,TCP_DW_PACKAGES_WITHOUTPL double,dummy_519 string,TCP_UL_RETRANS_WITHPL double,dummy_521 string,TCP_DW_RETRANS_WITHPL double,L4_UL_THROUGHPUT double,L4_DW_THROUGHPUT double,L4_UL_GOODPUT double,L4_DW_GOODPUT double,NETWORK_UL_TRAFFIC double,NETWORK_DL_TRAFFIC double,L4_UL_PACKETS double,L4_DW_PACKETS double,TCP_RTT double,TCP_UL_OUTOFSEQU double,TCP_DW_OUTOFSEQU double,TCP_UL_RETRANS double,TCP_DW_RETRANS double) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
-
-
-  }
-
-
-  //C20_SEQ_Dataload-01
-  test("C20_SEQ_Dataload-01", Include) {
-
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/SEQ500/seq_500Records.csv' into table smart_500_DE options('DELIMITER'=',', 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='SID,PROBEID,INTERFACEID,GROUPID,GGSN_ID,SGSN_ID,dummy,SESSION_INDICATOR,BEGIN_TIME,BEGIN_TIME_MSEL,END_TIME,END_TIME_MSEL,PROT_CATEGORY,PROT_TYPE,L7_CARRIER_PROT,SUB_PROT_TYPE,MSISDN,IMSI,IMEI,ENCRYPT_VERSION,ROAMING_TYPE,ROAM_DIRECTION,MS_IP,SERVER_IP,MS_PORT,APN,SGSN_SIG_IP,GGSN_USER_IP,SGSN_USER_IP,MCC,MNC,RAT,LAC,RAC,SAC,CI,SERVER_DECIMAL,BROWSER_TIMESTAMP,TCP_CONN_STATES,GGSN_SIG_IP_BigInt_NEGATIVE,TCP_STATES_BIGINTPOSITIVE,dummy,TCP_WIN_SIZE,dummy,TCP_MSS,dummy,TCP_CONN_TIMES,dummy,TCP_CONN_2_FAILED_TIMES,dummy,TCP_CONN_3_FAILED_TIMES,HOST,STREAMING_URL,dummy,GET_STREAMING_FAILED_CODE,dummy,GET_STREAMING_FLAG,dummy,GET_NUM,dummy,GET_SUCCEED_NUM,dummy,GET_RETRANS_NUM,dummy,GET_TIMEOUT_NUM,INTBUFFER_FST_FLAG,INTBUFFER_FULL_FLAG,STALL_NUM,dummy,VIDEO_FRAME_RATE,dummy,VIDEO_CODEC_ID,dummy,VIDEO_WIDTH,dummy,VIDEO_HEIGHT,dummy,AUDIO_CODEC_ID,dummy,MEDIA_FILE_TYPE,dummy,PLAY_STATE,dummy,PLAY_STATE,dummy,STREAMING_FLAG,dummy,TCP_STATUS_INDICATOR,dummy,DISCONNECTION_FLAG,dummy,FAILURE_CODE,FLAG,TAC,ECI,dummy,TCP_SYN_TIME_MSEL,dummy,TCP_FST_SYN_DIRECTION,RAN_NE_USER_IP,HOMEMCC,HOMEMNC,dummy,CHARGE_FLAG,dummy,PREPAID_FLAG,dummy,USER_AGENT,dummy,MS_WIN_STAT_TOTAL_NUM,dummy,MS_WIN_STAT_SMALL_NUM,dummy,MS_ACK_TO_1STGET_DELAY,dummy,SERVER_ACK_TO_1STDATA_DELAY,dummy,STREAMING_TYPE,dummy,SOURCE_VIDEO_QUALITY,TETHERING_FLAG,CARRIER_ID,LAYER1ID,LAYER2ID,dummy,LAYER3ID,dummy,LAYER4ID,dummy,LAYER5ID,dummy,LAYER6ID,CHARGING_RULE_BASE_NAME,SP,dummy,EXTENDED_URL,SV,FIRST_SAI_CGI_ECGI,dummy,EXTENDED_URL_OTHER,SIGNALING_USE_FLAG,dummy,DNS_RETRANS_NUM,dummy,DNS_FAIL_CODE,FIRST_RAT,FIRST_RAT,MS_INDICATOR,LAST_SAI_CGI_ECGI,LAST_RAT,dummy,FIRST_LONGITUDE,dummy,FIRST_LATITUDE,dummy,FIRST_ALTITUDE,dummy,FIRST_RASTERLONGITUDE,dummy,FIRST_RASTERLATITUDE,dummy,FIRST_RASTERALTITUDE,dummy,FIRST_FREQUENCYSPOT,dummy,FIRST_CLUTTER,dummy,FIRST_USERBEHAVIOR,dummy,FIRST_SPEED,dummy,FIRST_CREDIBILITY,dummy,LAST_LONGITUDE,dummy,LAST_LATITUDE,dummy,LAST_ALTITUDE,dummy,LAST_RASTERLONGITUDE,dummy,LAST_RASTERLATITUDE,dummy,LAST_RASTERALTITUDE,dummy,LAST_FREQUENCYSPOT,dummy,LAST_CLUTTER,dummy,LAST_USERBEHAVIOR,dummy,LAST_SPEED,dummy,LAST_CREDIBILITY,dummy,IMEI_CIPHERTEXT,APP_ID,dummy,DOMAIN_NAME,dummy,STREAMING_CACHE_IP,dummy,STOP_LONGER_THAN_MIN_THRESHOLD,dummy,STOP_LONGER_THAN_MAX_THRESHOLD,dummy,PLAY_END_STAT,dummy,STOP_START_TIME1,dummy,STOP_END_TIME1,dummy,STOP_START_TIME2,dummy,STOP_END_TIME2,dummy,STOP_START_TIME3,dummy,STOP_END_TIME3,dummy,STOP_START_TIME4,dummy,STOP_END_TIME4,dummy,STOP_START_TIME5,dummy,STOP_END_TIME5,dummy,STOP_START_TIME6,dummy,STOP_END_TIME6,dummy,STOP_START_TIME7,dummy,STOP_END_TIME7,dummy,STOP_START_TIME8,dummy,STOP_END_TIME8,dummy,STOP_START_TIME9,dummy,STOP_END_TIME9,dummy,STOP_START_TIME10,dummy,STOP_END_TIME10,dummy,FAIL_CLASS,RECORD_TYPE,dummy,NODATA_COUNT,dummy,VIDEO_NODATA_DURATION,dummy,VIDEO_SMOOTH_DURATION,dummy,VIDEO_SD_DURATION,dummy,VIDEO_HD_DURATION,dummy,VIDEO_UHD_DURATION,dummy,VIDEO_FHD_DURATION,dummy,FLUCTUATION,dummy,START_DOWNLOAD_THROUGHPUT,dummy,L7_UL_GOODPUT_FULL_MSS,dummy,SESSIONKEY,dummy,FIRST_UCELLID,dummy,LAST_UCELLID,dummy,UCELLID1,dummy,LONGITUDE1,dummy,LATITUDE1,dummy,UCELLID2,dummy,LONGITUDE2,dummy,LATITUDE2,dummy,UCELLID3,dummy,LONGITUDE3,dummy,LATITUDE3,dummy,UCELLID4,dummy,LONGITUDE4,dummy,LATITUDE4,dummy,UCELLID5,dummy,LONGITUDE5,dummy,LATITUDE5,dummy,UCELLID6,dummy,LONGITUDE6,dummy,LATITUDE6,dummy,UCELLID7,dummy,LONGITUDE7,dummy,LATITUDE7,dummy,UCELLID8,dummy,LONGITUDE8,dummy,LATITUDE8,dummy,UCELLID9,dummy,LONGITUDE9,dummy,LATITUDE9,dummy,UCELLID10,dummy,LONGITUDE10,dummy,LATITUDE10,dummy,INTBUFFER_FULL_DELAY,dummy,STALL_DURATION,dummy,STREAMING_DW_PACKETS,dummy,STREAMING_DOWNLOAD_DELAY,dummy,PLAY_DURATION,dummy,STREAMING_QUALITY,dummy,VIDEO_DATA_RATE,dummy,AUDIO_DATA_RATE,dummy,STREAMING_FILESIZE,dummy,STREAMING_DURATIOIN,dummy,TCP_SYN_TIME,dummy,TCP_RTT_STEP1,CHARGE_ID,dummy,UL_REVERSE_TO_DL_DELAY,dummy,DL_REVERSE_TO_UL_DELAY,dummy,DATATRANS_DW_GOODPUT,dummy,DATATRANS_DW_TOTAL_DURATION,dummy,SUM_FRAGMENT_INTERVAL,dummy,TCP_FIN_TIMES,dummy,TCP_RESET_TIMES,dummy,URL_CLASSIFICATION,dummy,STREAMING_LQ_DURATIOIN,dummy,MAX_DNS_DELAY,dummy,MAX_DNS2SYN,dummy,MAX_LATANCY_OF_LINK_SETUP,dummy,MAX_SYNACK2FIRSTACK,dummy,MAX_SYNACK2LASTACK,dummy,MAX_ACK2GET_DELAY,dummy,MAX_FRAG_INTERVAL_PREDELAY,dummy,SUM_FRAG_INTERVAL_PREDELAY,dummy,SERVICE_DELAY_MSEC,dummy,HOMEPROVINCE,dummy,HOMECITY,dummy,SERVICE_ID,dummy,CHARGING_CLASS,dummy,DATATRANS_UL_DURATION,dummy,ASSOCIATED_ID,dummy,PACKET_LOSS_NUM,dummy,JITTER,dummy,MS_DNS_DELAY_MSEL,dummy,GET_STREAMING_DELAY,dummy,TCP_UL_RETRANS_WITHOUTPL,dummy,TCP_DW_RETRANS_WITHOUTPL,dummy,GET_MAX_UL_SIZE,dummy,GET_MIN_UL_SIZE,dummy,GET_MAX_DL_SIZE,dummy,GET_MIN_DL_SIZE,dummy,FLOW_SAMPLE_RATIO,dummy,UL_RTT_LONG_NUM,dummy,DW_RTT_LONG_NUM,dummy,UL_RTT_STAT_NUM,dummy,DW_RTT_STAT_NUM,dummy,USER_PROBE_UL_LOST_PKT,dummy,SERVER_PROBE_UL_LOST_PKT,dummy,SERVER_PROBE_DW_LOST_PKT,dummy,USER_PROBE_DW_LOST_PKT,dummy,AVG_DW_RTT_MICRO_SEC,dummy,AVG_UL_RTT_MICRO_SEC,dummy,RAN_NE_ID,dummy,AVG_UL_RTT,dummy,AVG_DW_RTT,dummy,CHARGING_CHARACTERISTICS,dummy,DL_SERIOUS_OUT_OF_ORDER_NUM,dummy,DL_SLIGHT_OUT_OF_ORDER_NUM,dummy,DL_FLIGHT_TOTAL_SIZE,dummy,DL_FLIGHT_TOTAL_NUM,dummy,DL_MAX_FLIGHT_SIZE,dummy,VALID_TRANS_DURATION,dummy,AIR_PORT_DURATION,dummy,RADIO_CONN_TIMES,dummy,UL_SERIOUS_OUT_OF_ORDER_NUM,dummy,UL_SLIGHT_OUT_OF_ORDER_NUM,dummy,UL_FLIGHT_TOTAL_SIZE,dummy,UL_FLIGHT_TOTAL_NUM,dummy,UL_MAX_FLIGHT_SIZE,dummy,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS,dummy,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS,dummy,DL_CONTINUOUS_RETRANSMISSION_DELAY,dummy,USER_HUNGRY_DELAY,dummy,SERVER_HUNGRY_DELAY,dummy,UPPERLAYER_IP_UL_FRAGMENTS,dummy,UPPERLAYER_IP_DL_FRAGMENTS,dummy,DOWNLAYER_IP_UL_FRAGMENTS,dummy,DOWNLAYER_IP_DL_FRAGMENTS,dummy,UPPERLAYER_IP_UL_PACKETS,dummy,UPPERLAYER_IP_DL_PACKETS,dummy,DOWNLAYER_IP_UL_PACKETS,dummy,DOWNLAYER_IP_DL_PACKETS,dummy,TCP_UL_PACKAGES_WITHPL,dummy,TCP_DW_PACKAGES_WITHPL,dummy,TCP_UL_PACKAGES_WITHOUTPL,dummy,TCP_DW_PACKAGES_WITHOUTPL,dummy,TCP_UL_RETRANS_WITHPL,dummy,TCP_DW_RETRANS_WITHPL,L4_UL_THROUGHPUT,L4_DW_THROUGHPUT,L4_UL_GOODPUT,L4_DW_GOODPUT,NETWORK_UL_TRAFFIC,NETWORK_DL_TRAFFIC,L4_UL_PACKETS,L4_DW_PACKETS,TCP_RTT,TCP_UL_OUTOFSEQU,TCP_DW_OUTOFSEQU,TCP_UL_RETRANS,TCP_DW_RETRANS')""").collect
-
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/SEQ500/seq_500Records_hive1.csv' into table smart_500_DE_hive """).collect
-
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC001
-  test("C20_DICTIONARY_EXCLUDE_TC001", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DE where HOST not in ('www.hua735435.com')""",
-      s"""select SID, IMEI from smart_500_DE_hive where HOST not in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC001")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC002
-  test("C20_DICTIONARY_EXCLUDE_TC002", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DE where HOST in  ('www.hua735435.com')""",
-      s"""select SID, IMEI from smart_500_DE_hive where HOST in  ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC002")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC003
-  test("C20_DICTIONARY_EXCLUDE_TC003", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DE where HOST LIKE  'www.hua735435.com'""",
-      s"""select SID, IMEI from smart_500_DE_hive where HOST LIKE  'www.hua735435.com'""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC003")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC004
-  test("C20_DICTIONARY_EXCLUDE_TC004", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DE where HOST Not LIKE  'www.hua735435.com'""",
-      s"""select SID, IMEI from smart_500_DE_hive where HOST Not LIKE  'www.hua735435.com'""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC004")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC005
-  test("C20_DICTIONARY_EXCLUDE_TC005", Include) {
-
-    checkAnswer(s"""select length(HOST) from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST) from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC005")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC006
-  test("C20_DICTIONARY_EXCLUDE_TC006", Include) {
-
-    checkAnswer(s"""select avg(HOST),avg(LAYER1ID) from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select avg(HOST),avg(LAYER1ID) from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC006")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC007
-  test("C20_DICTIONARY_EXCLUDE_TC007", Include) {
-
-    checkAnswer(s"""select avg(HOST),avg(LAYER1ID) from smart_500_DE where HOST not in ('www.hua735435.com')""",
-      s"""select avg(HOST),avg(LAYER1ID) from smart_500_DE_hive where HOST not in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC007")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC008
-  test("C20_DICTIONARY_EXCLUDE_TC008", Include) {
-
-    checkAnswer(s"""select substring(IMEI,1,4) from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select substring(IMEI,1,4) from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC008")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC009
-  test("C20_DICTIONARY_EXCLUDE_TC009", Include) {
-
-    checkAnswer(s"""select length(HOST)+10 from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)+10 from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC009")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC010
-  test("C20_DICTIONARY_EXCLUDE_TC010", Include) {
-
-    checkAnswer(s"""select length(HOST)-10 from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)-10 from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC010")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC011
-  test("C20_DICTIONARY_EXCLUDE_TC011", Include) {
-
-    checkAnswer(s"""select length(HOST)/10 from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)/10 from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC011")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC012
-  test("C20_DICTIONARY_EXCLUDE_TC012", Include) {
-
-    checkAnswer(s"""select length(HOST)*10 from smart_500_DE where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)*10 from smart_500_DE_hive where HOST in ('www.hua735435.com')""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC012")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC013
-  test("C20_DICTIONARY_EXCLUDE_TC013", Include) {
-
-    checkAnswer(s"""select lower(MS_IP),sum(LAYER1ID) from smart_500_DE  group by lower(MS_IP)""",
-      s"""select lower(MS_IP),sum(LAYER1ID) from smart_500_DE_hive  group by lower(MS_IP)""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC013")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC014
-  test("C20_DICTIONARY_EXCLUDE_TC014", Include) {
-
-    checkAnswer(s"""select * from smart_500_DE  where unix_timestamp(MS_IP)=1420268400""",
-      s"""select * from smart_500_DE_hive  where unix_timestamp(MS_IP)=1420268400""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC014")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC015
-  test("C20_DICTIONARY_EXCLUDE_TC015", Include) {
-
-    checkAnswer(s"""select * from smart_500_DE  where to_date(MS_IP)='2015-01-07'""",
-      s"""select * from smart_500_DE_hive  where to_date(MS_IP)='2015-01-07'""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC015")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC016
-  test("C20_DICTIONARY_EXCLUDE_TC016", Include) {
-
-    checkAnswer(s"""select * from smart_500_DE  where datediff(MS_IP,'2014-12-01')>=35""",
-      s"""select * from smart_500_DE_hive  where datediff(MS_IP,'2014-12-01')>=35""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC016")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC017
-  test("C20_DICTIONARY_EXCLUDE_TC017", Include) {
-
-    checkAnswer(s"""select MS_IP,count(*) from smart_500_DE  group by MS_IP""",
-      s"""select MS_IP,count(*) from smart_500_DE_hive  group by MS_IP""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC017")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC018
-  test("C20_DICTIONARY_EXCLUDE_TC018", Include) {
-
-    sql(s"""select MS_IP,SID,count(*) from smart_500_DE  group by MS_IP,SID order by MS_IP limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC019
-  test("C20_DICTIONARY_EXCLUDE_TC019", Include) {
-
-    sql(s"""select SID,length( MSISDN),avg(LAYER1ID),avg(TCP_DW_RETRANS) from smart_500_DE  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC020
-  test("C20_DICTIONARY_EXCLUDE_TC020", Include) {
-
-    sql(s"""select SID,length( MSISDN),max(LAYER1ID),min(LAYER1ID) from smart_500_DE  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC021
-  test("C20_DICTIONARY_EXCLUDE_TC021", Include) {
-
-    sql(s"""select SID,length( MSISDN),max(LAYER1ID),max(LAYER1ID) from smart_500_DE  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC022
-  test("C20_DICTIONARY_EXCLUDE_TC022", Include) {
-
-    sql(s"""select SID,length( MSISDN),min(LAYER1ID),min(LAYER1ID) from smart_500_DE  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC023
-  test("C20_DICTIONARY_EXCLUDE_TC023", Include) {
-
-    sql(s"""select SID,length( MSISDN),max(LAYER1ID),min(LAYER1ID),avg(LAYER1ID) from smart_500_DE  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC024
-  test("C20_DICTIONARY_EXCLUDE_TC024", Include) {
-
-    checkAnswer(s"""select concat(upper(MSISDN),1),sum(LAYER1ID) from smart_500_DE  group by concat(upper(MSISDN),1)""",
-      s"""select concat(upper(MSISDN),1),sum(LAYER1ID) from smart_500_DE_hive  group by concat(upper(MSISDN),1)""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC024")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC025
-  test("C20_DICTIONARY_EXCLUDE_TC025", Include) {
-
-    checkAnswer(s"""select upper(substring(MSISDN,1,4)),sum(LAYER1ID) from smart_500_DE group by upper(substring(MSISDN,1,4)) """,
-      s"""select upper(substring(MSISDN,1,4)),sum(LAYER1ID) from smart_500_DE_hive group by upper(substring(MSISDN,1,4)) """, "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC025")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC026
-  test("C20_DICTIONARY_EXCLUDE_TC026", Include) {
-
-    checkAnswer(s"""select max(SERVER_IP) from smart_500_DE""",
-      s"""select max(SERVER_IP) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC026")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC027
-  test("C20_DICTIONARY_EXCLUDE_TC027", Include) {
-
-    checkAnswer(s"""select max(SERVER_IP+10) from smart_500_DE""",
-      s"""select max(SERVER_IP+10) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC027")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC028
-  test("C20_DICTIONARY_EXCLUDE_TC028", Include) {
-
-    checkAnswer(s"""select max(MSISDN) from smart_500_DE""",
-      s"""select max(MSISDN) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC028")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC029
-  test("C20_DICTIONARY_EXCLUDE_TC029", Include) {
-
-    checkAnswer(s"""select max(MSISDN+10) from smart_500_DE""",
-      s"""select max(MSISDN+10) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC029")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC030
-  test("C20_DICTIONARY_EXCLUDE_TC030", Include) {
-
-    checkAnswer(s"""select avg(TCP_DW_RETRANS) from smart_500_DE""",
-      s"""select avg(TCP_DW_RETRANS) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC030")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC031
-  test("C20_DICTIONARY_EXCLUDE_TC031", Include) {
-
-    checkAnswer(s"""select avg(TCP_DW_RETRANS+10) from smart_500_DE""",
-      s"""select avg(TCP_DW_RETRANS+10) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC031")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC032
-  test("C20_DICTIONARY_EXCLUDE_TC032", Include) {
-
-    checkAnswer(s"""select avg(TCP_DW_RETRANS-10) from smart_500_DE""",
-      s"""select avg(TCP_DW_RETRANS-10) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC032")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC033
-  test("C20_DICTIONARY_EXCLUDE_TC033", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS)-10 from smart_500_DE""",
-      s"""select count(TCP_DW_RETRANS)-10 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC033")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC034
-  test("C20_DICTIONARY_EXCLUDE_TC034", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS)-10 from smart_500_DE""",
-      s"""select count(TCP_DW_RETRANS)-10 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC034")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC035
-  test("C20_DICTIONARY_EXCLUDE_TC035", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS)-10 from smart_500_DE""",
-      s"""select count(TCP_DW_RETRANS)-10 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC035")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC036
-  test("C20_DICTIONARY_EXCLUDE_TC036", Include) {
-
-    checkAnswer(s"""select sum(MSISDN), sum(DISTINCT MSISDN) from smart_500_DE""",
-      s"""select sum(MSISDN), sum(DISTINCT MSISDN) from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC036")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC037
-  test("C20_DICTIONARY_EXCLUDE_TC037", Include) {
-
-    checkAnswer(s"""select count (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DE""",
-      s"""select count (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC037")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC038
-  test("C20_DICTIONARY_EXCLUDE_TC038", Include) {
-
-    checkAnswer(s"""select count (if(TCP_DW_RETRANS<100,NULL,TCP_DW_RETRANS))  a from smart_500_DE""",
-      s"""select count (if(TCP_DW_RETRANS<100,NULL,TCP_DW_RETRANS))  a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC038")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC039
-  test("C20_DICTIONARY_EXCLUDE_TC039", Include) {
-
-    checkAnswer(s"""select count (if(TCP_DW_RETRANS=100,NULL,TCP_DW_RETRANS))  a from smart_500_DE""",
-      s"""select count (if(TCP_DW_RETRANS=100,NULL,TCP_DW_RETRANS))  a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC039")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC040
-  test("C20_DICTIONARY_EXCLUDE_TC040", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS) from smart_500_DE where TCP_DW_RETRANS=100""",
-      s"""select count(TCP_DW_RETRANS) from smart_500_DE_hive where TCP_DW_RETRANS=100""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC040")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC041
-  test("C20_DICTIONARY_EXCLUDE_TC041", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS) from smart_500_DE where TCP_DW_RETRANS<100""",
-      s"""select count(TCP_DW_RETRANS) from smart_500_DE_hive where TCP_DW_RETRANS<100""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC041")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC042
-  test("C20_DICTIONARY_EXCLUDE_TC042", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS) from smart_500_DE where TCP_DW_RETRANS>100""",
-      s"""select count(TCP_DW_RETRANS) from smart_500_DE_hive where TCP_DW_RETRANS>100""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC042")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC043
-  test("C20_DICTIONARY_EXCLUDE_TC043", Include) {
-
-    sql(s"""select MSISDN, TCP_DW_RETRANS + LAYER1ID as a  from smart_500_DE order by MSISDN limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC044
-  test("C20_DICTIONARY_EXCLUDE_TC044", Include) {
-
-    sql(s"""select MSISDN, sum(TCP_DW_RETRANS + 10) Total from smart_500_DE group by  MSISDN order by Total limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC045
-  test("C20_DICTIONARY_EXCLUDE_TC045", Include) {
-
-    checkAnswer(s"""select MSISDN, min(LAYER1ID + 10)  Total from smart_500_DE group by  MSISDN order by MSISDN""",
-      s"""select MSISDN, min(LAYER1ID + 10)  Total from smart_500_DE_hive group by  MSISDN order by MSISDN""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC045")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC046
-  test("C20_DICTIONARY_EXCLUDE_TC046", Include) {
-
-    checkAnswer(s"""select avg (if(LAYER1ID>100,NULL,LAYER1ID))  a from smart_500_DE""",
-      s"""select avg (if(LAYER1ID>100,NULL,LAYER1ID))  a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC046")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC047
-  test("C20_DICTIONARY_EXCLUDE_TC047", Include) {
-
-    checkAnswer(s"""select avg (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DE""",
-      s"""select avg (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC047")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC048
-  ignore("C20_DICTIONARY_EXCLUDE_TC048", Include) {
-
-    checkAnswer(s"""select variance(LAYER1ID) as a   from smart_500_DE""",
-      s"""select variance(LAYER1ID) as a   from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC048")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC049
-  ignore("C20_DICTIONARY_EXCLUDE_TC049", Include) {
-
-    checkAnswer(s"""select var_pop(LAYER1ID)  as a from smart_500_DE""",
-      s"""select var_pop(LAYER1ID)  as a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC049")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC050
-  ignore("C20_DICTIONARY_EXCLUDE_TC050", Include) {
-
-    checkAnswer(s"""select var_samp(LAYER1ID) as a  from smart_500_DE""",
-      s"""select var_samp(LAYER1ID) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC050")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC051
-  ignore("C20_DICTIONARY_EXCLUDE_TC051", Include) {
-
-    checkAnswer(s"""select stddev_pop(LAYER1ID) as a  from smart_500_DE""",
-      s"""select stddev_pop(LAYER1ID) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC051")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC052
-  ignore("C20_DICTIONARY_EXCLUDE_TC052", Include) {
-
-    checkAnswer(s"""select stddev_samp(LAYER1ID)  as a from smart_500_DE""",
-      s"""select stddev_samp(LAYER1ID)  as a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC052")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC053
-  ignore("C20_DICTIONARY_EXCLUDE_TC053", Include) {
-
-    checkAnswer(s"""select covar_pop(LAYER1ID,LAYER1ID) as a  from smart_500_DE""",
-      s"""select covar_pop(LAYER1ID,LAYER1ID) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC053")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC054
-  ignore("C20_DICTIONARY_EXCLUDE_TC054", Include) {
-
-    checkAnswer(s"""select covar_samp(LAYER1ID,LAYER1ID) as a  from smart_500_DE""",
-      s"""select covar_samp(LAYER1ID,LAYER1ID) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC054")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC055
-  test("C20_DICTIONARY_EXCLUDE_TC055", Include) {
-
-    checkAnswer(s"""select corr(LAYER1ID,LAYER1ID)  as a from smart_500_DE""",
-      s"""select corr(LAYER1ID,LAYER1ID)  as a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC055")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC056
-  test("C20_DICTIONARY_EXCLUDE_TC056", Include) {
-
-    checkAnswer(s"""select percentile(LAYER1ID,0.2) as  a  from smart_500_DE""",
-      s"""select percentile(LAYER1ID,0.2) as  a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC056")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC057
-  test("C20_DICTIONARY_EXCLUDE_TC057", Include) {
-
-    checkAnswer(s"""select percentile(LAYER1ID,array(0,0.2,0.3,1))  as  a from smart_500_DE""",
-      s"""select percentile(LAYER1ID,array(0,0.2,0.3,1))  as  a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC057")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC058
-  test("C20_DICTIONARY_EXCLUDE_TC058", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,0.2) as a  from (select LAYER1ID from smart_500_DE order by LAYER1ID) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC059
-  test("C20_DICTIONARY_EXCLUDE_TC059", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,0.2,5) as a  from (select LAYER1ID from smart_500_DE order by LAYER1ID) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC060
-  test("C20_DICTIONARY_EXCLUDE_TC060", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,array(0.2,0.3,0.99))  as a from (select LAYER1ID from smart_500_DE order by LAYER1ID) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC061
-  test("C20_DICTIONARY_EXCLUDE_TC061", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,array(0.2,0.3,0.99),5) as a from (select LAYER1ID from smart_500_DE order by LAYER1ID) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC062
-  test("C20_DICTIONARY_EXCLUDE_TC062", Include) {
-
-    sql(s"""select histogram_numeric(LAYER1ID,2)  as a from (select LAYER1ID from smart_500_DE order by LAYER1ID) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC063
-  ignore("C20_DICTIONARY_EXCLUDE_TC063", Include) {
-
-    checkAnswer(s"""select variance(TCP_DW_RETRANS) as a from (select * from smart_500_DE order by TCP_DW_RETRANS) t""",
-      s"""select variance(TCP_DW_RETRANS) as a from (select * from smart_500_DE_hive order by TCP_DW_RETRANS) t""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC063")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC064
-  ignore("C20_DICTIONARY_EXCLUDE_TC064", Include) {
-
-    checkAnswer(s"""select var_pop(TCP_DW_RETRANS)  as a from smart_500_DE""",
-      s"""select var_pop(TCP_DW_RETRANS)  as a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC064")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC065
-  ignore("C20_DICTIONARY_EXCLUDE_TC065", Include) {
-
-    checkAnswer(s"""select var_samp(TCP_DW_RETRANS) as a  from smart_500_DE""",
-      s"""select var_samp(TCP_DW_RETRANS) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC065")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC066
-  ignore("C20_DICTIONARY_EXCLUDE_TC066", Include) {
-
-    checkAnswer(s"""select stddev_pop(TCP_DW_RETRANS) as a  from smart_500_DE""",
-      s"""select stddev_pop(TCP_DW_RETRANS) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC066")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC067
-  ignore("C20_DICTIONARY_EXCLUDE_TC067", Include) {
-
-    checkAnswer(s"""select stddev_samp(TCP_DW_RETRANS)  as a from smart_500_DE""",
-      s"""select stddev_samp(TCP_DW_RETRANS)  as a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC067")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC068
-  ignore("C20_DICTIONARY_EXCLUDE_TC068", Include) {
-
-    checkAnswer(s"""select covar_pop(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DE""",
-      s"""select covar_pop(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC068")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC069
-  ignore("C20_DICTIONARY_EXCLUDE_TC069", Include) {
-
-    checkAnswer(s"""select covar_samp(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DE""",
-      s"""select covar_samp(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC069")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC070
-  test("C20_DICTIONARY_EXCLUDE_TC070", Include) {
-
-    checkAnswer(s"""select corr(TCP_DW_RETRANS,TCP_DW_RETRANS)  as a from smart_500_DE""",
-      s"""select corr(TCP_DW_RETRANS,TCP_DW_RETRANS)  as a from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC070")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC073
-  test("C20_DICTIONARY_EXCLUDE_TC073", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,0.2) as a  from (select TCP_DW_RETRANS from smart_500_DE order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC074
-  test("C20_DICTIONARY_EXCLUDE_TC074", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,0.2,5) as a  from (select TCP_DW_RETRANS from smart_500_DE order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC075
-  test("C20_DICTIONARY_EXCLUDE_TC075", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,array(0.2,0.3,0.99))  as a from (select TCP_DW_RETRANS from smart_500_DE order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC076
-  test("C20_DICTIONARY_EXCLUDE_TC076", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,array(0.2,0.3,0.99),5) as a from (select TCP_DW_RETRANS from smart_500_DE order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC077
-  test("C20_DICTIONARY_EXCLUDE_TC077", Include) {
-
-    sql(s"""select histogram_numeric(TCP_DW_RETRANS,2)  as a from (select TCP_DW_RETRANS from smart_500_DE order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC094
-  test("C20_DICTIONARY_EXCLUDE_TC094", Include) {
-
-    sql(s"""select Upper(streaming_url) a ,host from smart_500_DE order by host limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC095
-  test("C20_DICTIONARY_EXCLUDE_TC095", Include) {
-
-    sql(s"""select Lower(streaming_url) a  from smart_500_DE order by host limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC096
-  test("C20_DICTIONARY_EXCLUDE_TC096", Include) {
-
-    sql(s"""select streaming_url as b,LAYER1ID as a from smart_500_DE  order by a,b asc limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC097
-  test("C20_DICTIONARY_EXCLUDE_TC097", Include) {
-
-    sql(s"""select streaming_url as b,TCP_DW_RETRANS as a from smart_500_DE  order by a,b desc limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC098
-  test("C20_DICTIONARY_EXCLUDE_TC098", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where streaming_url ='www.hua1/xyz'""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where streaming_url ='www.hua1/xyz'""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC098")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC099
-  test("C20_DICTIONARY_EXCLUDE_TC099", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where streaming_url ='www.hua90/xyz' and TCP_DW_RETRANS ='82.0' limit 10""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where streaming_url ='www.hua90/xyz' and TCP_DW_RETRANS ='82.0' limit 10""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC099")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC100
-  test("C20_DICTIONARY_EXCLUDE_TC100", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where streaming_url ='www.hua1/xyz' or  TCP_DW_RETRANS ='82.0'""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where streaming_url ='www.hua1/xyz' or  TCP_DW_RETRANS ='82.0'""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC100")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC101
-  test("C20_DICTIONARY_EXCLUDE_TC101", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where streaming_url !='www.hua1/xyz'  order by LAYER1ID,TCP_DW_RETRANS,streaming_url""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where streaming_url !='www.hua1/xyz'  order by LAYER1ID,TCP_DW_RETRANS,streaming_url""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC101")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC102
-  test("C20_DICTIONARY_EXCLUDE_TC102", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where streaming_url !='www.hua1/xyz' and TCP_DW_RETRANS !='152.0'  order by LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where streaming_url !='www.hua1/xyz' and TCP_DW_RETRANS !='152.0'  order by LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC102")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC103
-  test("C20_DICTIONARY_EXCLUDE_TC103", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where TCP_DW_RETRANS >2.0 order by LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where TCP_DW_RETRANS >2.0 order by LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC103")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC104
-  test("C20_DICTIONARY_EXCLUDE_TC104", Include) {
-
-    sql(s"""select LAYER1ID as a from smart_500_DE where LAYER1ID<=>LAYER1ID order by LAYER1ID desc limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC105
-  test("C20_DICTIONARY_EXCLUDE_TC105", Include) {
-
-    sql(s"""SELECT LAYER1ID,TCP_DW_RETRANS,streaming_url FROM (select * from smart_500_DE) SUB_QRY ORDER BY LAYER1ID,TCP_DW_RETRANS,streaming_url ASC limit 10""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC106
-  test("C20_DICTIONARY_EXCLUDE_TC106", Include) {
-
-    sql(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where ( LAYER1ID+1) == 101 order by TCP_DW_RETRANS,LAYER1ID limit 5""").collect
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC107
-  test("C20_DICTIONARY_EXCLUDE_TC107", Include) {
-
-    checkAnswer(s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where  streaming_url is  null order by LAYER1ID,TCP_DW_RETRANS,streaming_url                                """,
-      s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where  streaming_url is  null order by LAYER1ID,TCP_DW_RETRANS,streaming_url                                """, "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC107")
-
-  }
-
-
-  //C20_DICTIONARY_EXCLUDE_TC108
-  test("C20_DICTIONARY_EXCLUDE_TC108", Include) {
-
-    checkAnswer(s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE where  streaming_url is  not null order by LAYER1ID,TCP_DW_RETRANS,streaming_url""",
-      s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DE_hive where  streaming_url is  not null order by LAYER1ID,TCP_DW_RETRANS,streaming_url""", "QueriesExcludeDictionaryTestCase_C20_DICTIONARY_EXCLUDE_TC108")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC001
-  test("PushUP_FILTER_smart_500_DE_TC001", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN=17846415579 or  IMSI=460075195040377 or BEGIN_TIME=1.463483694712E12""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN=17846415579 or  IMSI=460075195040377 or BEGIN_TIME=1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC001")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC002
-  test("PushUP_FILTER_smart_500_DE_TC002", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN=17846602163 and  BEGIN_TIME=1.463483694712E12 and SERVER_IP='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN=17846602163 and  BEGIN_TIME=1.463483694712E12 and SERVER_IP='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC002")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC003
-  test("PushUP_FILTER_smart_500_DE_TC003", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where (MSISDN==17846602163) and  (BEGIN_TIME=1.463483694712E12) and (SERVER_IP=='192.26.210.204')""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where (MSISDN==17846602163) and  (BEGIN_TIME=1.463483694712E12) and (SERVER_IP=='192.26.210.204')""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC003")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC004
-  test("PushUP_FILTER_smart_500_DE_TC004", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where (MSISDN==17846415579) or  (IMSI==460075195040377) or (BEGIN_TIME==1.463483694712E12)""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where (MSISDN==17846415579) or  (IMSI==460075195040377) or (BEGIN_TIME==1.463483694712E12)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC004")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC005
-  test("PushUP_FILTER_smart_500_DE_TC005", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where (MSISDN==17846602163) and  (BEGIN_TIME==1.463483694712E12) or (SERVER_IP=='192.26.210.204')""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where (MSISDN==17846602163) and  (BEGIN_TIME==1.463483694712E12) or (SERVER_IP=='192.26.210.204')""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC005")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC006
-  test("PushUP_FILTER_smart_500_DE_TC006", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN!=17846602163 and  BEGIN_TIME!=1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN!=17846602163 and  BEGIN_TIME!=1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC006")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC007
-  test("PushUP_FILTER_smart_500_DE_TC007", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN!=17846602163 or  BEGIN_TIME!=1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN!=17846602163 or  BEGIN_TIME!=1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC007")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC008
-  test("PushUP_FILTER_smart_500_DE_TC008", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN!=17846602163 and  BEGIN_TIME!=1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN!=17846602163 and  BEGIN_TIME!=1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC008")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC009
-  test("PushUP_FILTER_smart_500_DE_TC009", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN IS NOT NULL and  BEGIN_TIME IS NOT NULL and SERVER_IP IS NOT NULL""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN IS NOT NULL and  BEGIN_TIME IS NOT NULL and SERVER_IP IS NOT NULL""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC009")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC010
-  test("PushUP_FILTER_smart_500_DE_TC010", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN IS NOT NULL or  BEGIN_TIME IS NOT NULL or SERVER_IP IS NOT NULL""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN IS NOT NULL or  BEGIN_TIME IS NOT NULL or SERVER_IP IS NOT NULL""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC010")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC011
-  test("PushUP_FILTER_smart_500_DE_TC011", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN NOT IN (17846415579,17846415580) and  IMSI NOT IN (460075195040377,460075195040378) or BEGIN_TIME NOT IN (1.463483694712E12,1.463483694712E13)""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN NOT IN (17846415579,17846415580) and  IMSI NOT IN (460075195040377,460075195040378) or BEGIN_TIME NOT IN (1.463483694712E12,1.463483694712E13)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC011")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC012
-  test("PushUP_FILTER_smart_500_DE_TC012", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN IN (17846415579,17846415580) and  IMSI IN (460075195040377,460075195040378) or BEGIN_TIME IN (1.463483694712E12,1.463483694712E13)""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN IN (17846415579,17846415580) and  IMSI IN (460075195040377,460075195040378) or BEGIN_TIME IN (1.463483694712E12,1.463483694712E13)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC012")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC013
-  test("PushUP_FILTER_smart_500_DE_TC013", Include) {
-
-    checkAnswer(s"""select MSISDN+0.1000001,IMSI+9999999,BEGIN_TIME+9.999999 from smart_500_DE""",
-      s"""select MSISDN+0.1000001,IMSI+9999999,BEGIN_TIME+9.999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC013")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC014
-  test("PushUP_FILTER_smart_500_DE_TC014", Include) {
-
-    checkAnswer(s"""select MSISDN-0.1000001,IMSI-9999999,BEGIN_TIME-9.999999 from smart_500_DE""",
-      s"""select MSISDN-0.1000001,IMSI-9999999,BEGIN_TIME-9.999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC014")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC015
-  test("PushUP_FILTER_smart_500_DE_TC015", Include) {
-
-    checkAnswer(s"""select MSISDN*0.1000001,IMSI*9999999,BEGIN_TIME*9.999999 from smart_500_DE""",
-      s"""select MSISDN*0.1000001,IMSI*9999999,BEGIN_TIME*9.999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC015")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC016
-  test("PushUP_FILTER_smart_500_DE_TC016", Include) {
-
-    checkAnswer(s"""select MSISDN/0.1000001,IMSI/9999999,BEGIN_TIME/9.999999 from smart_500_DE""",
-      s"""select MSISDN/0.1000001,IMSI/9999999,BEGIN_TIME/9.999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC016")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC017
-  test("PushUP_FILTER_smart_500_DE_TC017", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN>17846602163 and  BEGIN_TIME>1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN>17846602163 and  BEGIN_TIME>1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC017")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC018
-  test("PushUP_FILTER_smart_500_DE_TC018", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN<17846602163 and  BEGIN_TIME<1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN<17846602163 and  BEGIN_TIME<1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC018")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC019
-  test("PushUP_FILTER_smart_500_DE_TC019", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN>=17846602163 and  BEGIN_TIME>=1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN>=17846602163 and  BEGIN_TIME>=1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC019")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC020
-  test("PushUP_FILTER_smart_500_DE_TC020", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN<=17846602163 and  BEGIN_TIME<=1.463483694712E12 or SERVER_IP!='192.26.210.204'""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN<=17846602163 and  BEGIN_TIME<=1.463483694712E12 or SERVER_IP!='192.26.210.204'""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC020")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC021
-  test("PushUP_FILTER_smart_500_DE_TC021", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN like 17846415579 and  IMSI like 460075195040377 or BEGIN_TIME like 1.463483694712E12""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN like 17846415579 and  IMSI like 460075195040377 or BEGIN_TIME like 1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC021")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC022
-  test("PushUP_FILTER_smart_500_DE_TC022", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,END_TIME from smart_500_DE where MSISDN not like 17846415579 and  IMSI not like 460075195040377 or BEGIN_TIME not like 1.463483694712E12""",
-      s"""select BEGIN_TIME,END_TIME from smart_500_DE_hive where MSISDN not like 17846415579 and  IMSI not like 460075195040377 or BEGIN_TIME not like 1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC022")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC023
-  test("PushUP_FILTER_smart_500_DE_TC023", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID=4.61168620184322E14 or  IMSI=460075195040377 or BEGIN_TIME=1.463483694712E12""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID=4.61168620184322E14 or  IMSI=460075195040377 or BEGIN_TIME=1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC023")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC024
-  test("PushUP_FILTER_smart_500_DE_TC024", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID=4.61168620184322E14 and  IMSI=460075195040377 and BEGIN_TIME=1.463483694712E12""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID=4.61168620184322E14 and  IMSI=460075195040377 and BEGIN_TIME=1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC024")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC025
-  test("PushUP_FILTER_smart_500_DE_TC025", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where (SID==4.61168620184322E14) and  (IMSI==460075171072129) and (BEGIN_TIME==1.463483694712E12)""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where (SID==4.61168620184322E14) and  (IMSI==460075171072129) and (BEGIN_TIME==1.463483694712E12)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC025")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC026
-  test("PushUP_FILTER_smart_500_DE_TC026", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where (SID==4.61168620184322E14) or  (IMSI==460075171072129) or (BEGIN_TIME==1.463483694712E12)""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where (SID==4.61168620184322E14) or  (IMSI==460075171072129) or (BEGIN_TIME==1.463483694712E12)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC026")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC027
-  test("PushUP_FILTER_smart_500_DE_TC027", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where (SID==4.61168620184322E14) and  (IMSI==460075171072129) or (BEGIN_TIME==1.463483694712E12)""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where (SID==4.61168620184322E14) and  (IMSI==460075171072129) or (BEGIN_TIME==1.463483694712E12)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC027")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC028
-  test("PushUP_FILTER_smart_500_DE_TC028", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID!=4.61168620184322E14 or  IMSI!=460075171072129 or BEGIN_TIME!=1.463483694712E12""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID!=4.61168620184322E14 or  IMSI!=460075171072129 or BEGIN_TIME!=1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC028")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC029
-  test("PushUP_FILTER_smart_500_DE_TC029", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID!=4.61168620184322E14 and  IMSI!=460075171072129 or BEGIN_TIME!=1.463483694712E12""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID!=4.61168620184322E14 and  IMSI!=460075171072129 or BEGIN_TIME!=1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC029")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC030
-  test("PushUP_FILTER_smart_500_DE_TC030", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID is NOT NULL and  IMSI is not null and BEGIN_TIME is not null""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID is NOT NULL and  IMSI is not null and BEGIN_TIME is not null""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC030")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC031
-  test("PushUP_FILTER_smart_500_DE_TC031", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID is NOT NULL or  IMSI is not null or BEGIN_TIME is not null""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID is NOT NULL or  IMSI is not null or BEGIN_TIME is not null""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC031")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC032
-  test("PushUP_FILTER_smart_500_DE_TC032", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID NOT IN (4.61168620184322E14,4.61168620184322E16) or  IMSI NOT IN (460075171072129,460075171072130) or BEGIN_TIME NOT IN (1.463483694712E12,1.463483694712E14)""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID NOT IN (4.61168620184322E14,4.61168620184322E16) or  IMSI NOT IN (460075171072129,460075171072130) or BEGIN_TIME NOT IN (1.463483694712E12,1.463483694712E14)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC032")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC033
-  test("PushUP_FILTER_smart_500_DE_TC033", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID IN (4.61168620184322E14,4.61168620184322E16) or  IMSI IN (460075171072129,460075171072130) or BEGIN_TIME IN (1.463483694712E12,1.463483694712E14)""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID IN (4.61168620184322E14,4.61168620184322E16) or  IMSI IN (460075171072129,460075171072130) or BEGIN_TIME IN (1.463483694712E12,1.463483694712E14)""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC033")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC034
-  test("PushUP_FILTER_smart_500_DE_TC034", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME+2,SID+0.234,IMSI+99.99999999 from smart_500_DE""",
-      s"""select BEGIN_TIME+2,SID+0.234,IMSI+99.99999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC034")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC035
-  test("PushUP_FILTER_smart_500_DE_TC035", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME-2,SID-0.234,IMSI-99.99999999 from smart_500_DE""",
-      s"""select BEGIN_TIME-2,SID-0.234,IMSI-99.99999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC035")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC036
-  test("PushUP_FILTER_smart_500_DE_TC036", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME*2,SID*0.234,IMSI*99.99999999 from smart_500_DE""",
-      s"""select BEGIN_TIME*2,SID*0.234,IMSI*99.99999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC036")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC037
-  test("PushUP_FILTER_smart_500_DE_TC037", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME/2,SID/0.234,IMSI/99.99999999 from smart_500_DE""",
-      s"""select BEGIN_TIME/2,SID/0.234,IMSI/99.99999999 from smart_500_DE_hive""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC037")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC038
-  test("PushUP_FILTER_smart_500_DE_TC038", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID like 4.61168620184322E14 and  IMSI like 460075171072129  or BEGIN_TIME like 1.463483694712E12""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID like 4.61168620184322E14 and  IMSI like 460075171072129  or BEGIN_TIME like 1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC038")
-
-  }
-
-
-  //PushUP_FILTER_smart_500_DE_TC039
-  test("PushUP_FILTER_smart_500_DE_TC039", Include) {
-
-    checkAnswer(s"""select BEGIN_TIME,SID from smart_500_DE where SID not like 4.61168620184322E14 and  IMSI not like 460075171072129  or BEGIN_TIME not like 1.463483694712E12""",
-      s"""select BEGIN_TIME,SID from smart_500_DE_hive where SID not like 4.61168620184322E14 and  IMSI not like 460075171072129  or BEGIN_TIME not like 1.463483694712E12""", "QueriesExcludeDictionaryTestCase_PushUP_FILTER_smart_500_DE_TC039")
-
-  }
-
-  override def afterAll {
-  sql("drop table if exists TABLE_DICTIONARY_EXCLUDE")
-  sql("drop table if exists TABLE_DICTIONARY_EXCLUDE1_hive")
-  sql("drop table if exists TABLE_DICTIONARY_EXCLUDE1")
-  sql("drop table if exists TABLE_DICTIONARY_EXCLUDE1_hive")
-  sql("drop table if exists smart_500_DE")
-  sql("drop table if exists smart_500_DE_hive")
-  }
-}
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesIncludeDictionaryTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesIncludeDictionaryTestCase.scala
deleted file mode 100644
index dad731f..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesIncludeDictionaryTestCase.scala
+++ /dev/null
@@ -1,4686 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import org.apache.spark.sql.common.util._
-import org.scalatest.BeforeAndAfterAll
-
-/**
- * Test Class for QueriesIncludeDictionaryTestCase to verify all scenerios
- */
-
-class QueriesIncludeDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
-         
-
-  //TABLE_DICTIONARY_INCLUDE_CreateCube
-  test("TABLE_DICTIONARY_INCLUDE_CreateCube", Include) {
-    sql(s"""drop table if exists TABLE_DICTIONARY_INCLUDE""").collect
-    sql(s"""drop table if exists TABLE_DICTIONARY_INCLUDE_hive""").collect
-
-    sql(s"""create table  TABLE_DICTIONARY_INCLUDE (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='imei,deviceInformationId,productionDate,gamePointId,Latest_DAY,contractNumber')
-  """).collect
-
-    sql(s"""create table  TABLE_DICTIONARY_INCLUDE_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string,deliveryTime string,channelsId string,channelsName string,deliveryAreaId string,deliveryCountry string,deliveryProvince string,deliveryCity string,deliveryDistrict string,deliveryStreet string,oxSingleNumber string,contractNumber BigInt,ActiveCheckTime string,ActiveAreaId string,ActiveCountry string,ActiveProvince string,Activecity string,ActiveDistrict string,ActiveStreet string,ActiveOperatorId string,Active_releaseId string,Active_EMUIVersion string,Active_operaSysVersion string,Active_BacVerNumber string,Active_BacFlashVer string,Active_webUIVersion string,Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string,Active_operatorsVersion string,Active_phonePADPartitionedVersions string,Latest_YEAR int,Latest_MONTH int,Latest_DAY Decimal(30,10),Latest_HOUR string,Latest_areaId string,Latest_country string,Latest_province string,Latest_city string,Latest_district string,Latest_street string,Latest_releaseId string,Latest_EMUIVersion string,Latest_operaSysVersion string,Latest_BacVerNumber string,Latest_BacFlashVer string,Latest_webUIVersion string,Latest_webUITypeCarrVer string,Latest_webTypeDataVerNumber string,Latest_operatorsVersion string,Latest_phonePADPartitionedVersions string,Latest_operatorId string,gamePointId double,gamePointDescription string) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
-
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_CreateCube_count
-  test("TABLE_DICTIONARY_INCLUDE_CreateCube_count", Include) {
-
-    sql(s"""select count(*) from TABLE_DICTIONARY_INCLUDE""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_DataLoad
-  test("TABLE_DICTIONARY_INCLUDE_DataLoad", Include) {
-
-    sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/100_olap_C20.csv' INTO table TABLE_DICTIONARY_INCLUDE options ('DELIMITER'=',', 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
-
-    sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/100_olap_C20_hive42.csv' INTO table TABLE_DICTIONARY_INCLUDE_hive """).collect
-
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_001
-  test("TABLE_DICTIONARY_INCLUDE_001", Include) {
-
-    checkAnswer(s"""Select count(imei) from TABLE_DICTIONARY_INCLUDE""",
-      s"""Select count(imei) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_001")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_002
-  test("TABLE_DICTIONARY_INCLUDE_002", Include) {
-
-    checkAnswer(s"""select count(DISTINCT imei) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT imei) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_002")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_003
-  test("TABLE_DICTIONARY_INCLUDE_003", Include) {
-
-    checkAnswer(s"""select sum(Latest_month)+10 as a ,imei  from TABLE_DICTIONARY_INCLUDE group by imei order by imei""",
-      s"""select sum(Latest_month)+10 as a ,imei  from TABLE_DICTIONARY_INCLUDE_hive group by imei order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_003")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_004
-  test("TABLE_DICTIONARY_INCLUDE_004", Include) {
-
-    checkAnswer(s"""select max(imei),min(imei) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(imei),min(imei) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_004")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_005
-  test("TABLE_DICTIONARY_INCLUDE_005", Include) {
-
-    checkAnswer(s"""select min(imei), max(imei) Total from TABLE_DICTIONARY_INCLUDE group by  channelsId order by Total""",
-      s"""select min(imei), max(imei) Total from TABLE_DICTIONARY_INCLUDE_hive group by  channelsId order by Total""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_005")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_006
-  test("TABLE_DICTIONARY_INCLUDE_006", Include) {
-
-    checkAnswer(s"""select last(imei) a from TABLE_DICTIONARY_INCLUDE  group by imei order by imei limit 1""",
-      s"""select last(imei) a from TABLE_DICTIONARY_INCLUDE_hive  group by imei order by imei limit 1""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_006")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_007
-  test("TABLE_DICTIONARY_INCLUDE_007", Include) {
-
-    sql(s"""select FIRST(imei) a from TABLE_DICTIONARY_INCLUDE group by imei order by imei limit 1""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_008
-  test("TABLE_DICTIONARY_INCLUDE_008", Include) {
-
-    checkAnswer(s"""select imei,count(imei) a from TABLE_DICTIONARY_INCLUDE group by imei order by imei""",
-      s"""select imei,count(imei) a from TABLE_DICTIONARY_INCLUDE_hive group by imei order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_008")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_009
-  test("TABLE_DICTIONARY_INCLUDE_009", Include) {
-
-    checkAnswer(s"""select Lower(imei) a  from TABLE_DICTIONARY_INCLUDE order by imei""",
-      s"""select Lower(imei) a  from TABLE_DICTIONARY_INCLUDE_hive order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_009")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_010
-  test("TABLE_DICTIONARY_INCLUDE_010", Include) {
-
-    checkAnswer(s"""select distinct imei from TABLE_DICTIONARY_INCLUDE order by imei""",
-      s"""select distinct imei from TABLE_DICTIONARY_INCLUDE_hive order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_010")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_011
-  test("TABLE_DICTIONARY_INCLUDE_011", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE order by imei limit 101 """,
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive order by imei limit 101 """, "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_011")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_012
-  test("TABLE_DICTIONARY_INCLUDE_012", Include) {
-
-    checkAnswer(s"""select imei as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select imei as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_012")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_013
-  test("TABLE_DICTIONARY_INCLUDE_013", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where  (contractNumber == 9223372047700) and (imei=='1AA100004')""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where  (contractNumber == 9223372047700) and (imei=='1AA100004')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_013")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_014
-  test("TABLE_DICTIONARY_INCLUDE_014", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei !='1AA100064' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei !='1AA100064' order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_014")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_015
-  test("TABLE_DICTIONARY_INCLUDE_015", Include) {
-
-    checkAnswer(s"""select imei  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select imei  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_015")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_016
-  test("TABLE_DICTIONARY_INCLUDE_016", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei !='1AA100012' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei !='1AA100012' order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_016")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_017
-  test("TABLE_DICTIONARY_INCLUDE_017", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei >'1AA100012' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei >'1AA100012' order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_017")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_018
-  test("TABLE_DICTIONARY_INCLUDE_018", Include) {
-
-    checkAnswer(s"""select imei  from TABLE_DICTIONARY_INCLUDE where imei<>imei""",
-      s"""select imei  from TABLE_DICTIONARY_INCLUDE_hive where imei<>imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_018")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_019
-  test("TABLE_DICTIONARY_INCLUDE_019", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei != Latest_areaId order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei != Latest_areaId order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_019")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_020
-  test("TABLE_DICTIONARY_INCLUDE_020", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where Latest_areaId<imei order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where Latest_areaId<imei order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_020")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_021
-  test("TABLE_DICTIONARY_INCLUDE_021", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where Latest_DAY<=imei order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY<=imei order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_021")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_022
-  test("TABLE_DICTIONARY_INCLUDE_022", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei <'1AA10002' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei <'1AA10002' order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_022")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_023
-  test("TABLE_DICTIONARY_INCLUDE_023", Include) {
-
-    checkAnswer(s"""select Latest_day  from TABLE_DICTIONARY_INCLUDE where imei IS NULL""",
-      s"""select Latest_day  from TABLE_DICTIONARY_INCLUDE_hive where imei IS NULL""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_023")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_024
-  test("TABLE_DICTIONARY_INCLUDE_024", Include) {
-
-    checkAnswer(s"""select Latest_day  from TABLE_DICTIONARY_INCLUDE where imei IS NOT NULL order by Latest_day""",
-      s"""select Latest_day  from TABLE_DICTIONARY_INCLUDE_hive where imei IS NOT NULL order by Latest_day""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_024")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_025
-  test("TABLE_DICTIONARY_INCLUDE_025", Include) {
-
-    checkAnswer(s"""Select count(imei),min(imei) from TABLE_DICTIONARY_INCLUDE """,
-      s"""Select count(imei),min(imei) from TABLE_DICTIONARY_INCLUDE_hive """, "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_025")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_026
-  test("TABLE_DICTIONARY_INCLUDE_026", Include) {
-
-    checkAnswer(s"""select count(DISTINCT imei,latest_day) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT imei,latest_day) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_026")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_027
-  test("TABLE_DICTIONARY_INCLUDE_027", Include) {
-
-    checkAnswer(s"""select max(imei),min(imei),count(imei) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(imei),min(imei),count(imei) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_027")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_028
-  test("TABLE_DICTIONARY_INCLUDE_028", Include) {
-
-    checkAnswer(s"""select sum(imei),avg(imei),count(imei) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(imei),avg(imei),count(imei) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_028")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_029
-  test("TABLE_DICTIONARY_INCLUDE_029", Include) {
-
-    sql(s"""select last(imei),Min(imei),max(imei)  a from (select imei from TABLE_DICTIONARY_INCLUDE order by imei) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_030
-  test("TABLE_DICTIONARY_INCLUDE_030", Include) {
-
-    sql(s"""select FIRST(imei),Last(imei) a from TABLE_DICTIONARY_INCLUDE group by imei order by imei limit 1""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_031
-  test("TABLE_DICTIONARY_INCLUDE_031", Include) {
-
-    checkAnswer(s"""select imei,count(imei) a from TABLE_DICTIONARY_INCLUDE group by imei order by imei""",
-      s"""select imei,count(imei) a from TABLE_DICTIONARY_INCLUDE_hive group by imei order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_031")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_032
-  test("TABLE_DICTIONARY_INCLUDE_032", Include) {
-
-    checkAnswer(s"""select Lower(imei),upper(imei)  a  from TABLE_DICTIONARY_INCLUDE order by imei""",
-      s"""select Lower(imei),upper(imei)  a  from TABLE_DICTIONARY_INCLUDE_hive order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_032")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_033
-  test("TABLE_DICTIONARY_INCLUDE_033", Include) {
-
-    checkAnswer(s"""select imei as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select imei as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_033")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_034
-  test("TABLE_DICTIONARY_INCLUDE_034", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where  (contractNumber == 9223372047700) and (imei=='1AA100012')""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where  (contractNumber == 9223372047700) and (imei=='1AA100012')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_034")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_035
-  test("TABLE_DICTIONARY_INCLUDE_035", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei !='8imei' order by imei""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei !='8imei' order by imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_035")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_036
-  test("TABLE_DICTIONARY_INCLUDE_036", Include) {
-
-    checkAnswer(s"""select imei  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select imei  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_036")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_037
-  test("TABLE_DICTIONARY_INCLUDE_037", Include) {
-
-    checkAnswer(s"""Select count(contractNumber) from TABLE_DICTIONARY_INCLUDE""",
-      s"""Select count(contractNumber) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_037")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_038
-  test("TABLE_DICTIONARY_INCLUDE_038", Include) {
-
-    checkAnswer(s"""select count(DISTINCT contractNumber) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT contractNumber) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_038")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_039
-  test("TABLE_DICTIONARY_INCLUDE_039", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)+10 as a ,contractNumber  from TABLE_DICTIONARY_INCLUDE group by contractNumber""",
-      s"""select sum(contractNumber)+10 as a ,contractNumber  from TABLE_DICTIONARY_INCLUDE_hive group by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_039")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_040
-  test("TABLE_DICTIONARY_INCLUDE_040", Include) {
-
-    checkAnswer(s"""select max(contractNumber),min(contractNumber) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(contractNumber),min(contractNumber) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_040")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_041
-  test("TABLE_DICTIONARY_INCLUDE_041", Include) {
-
-    checkAnswer(s"""select sum(contractNumber) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(contractNumber) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_041")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_042
-  test("TABLE_DICTIONARY_INCLUDE_042", Include) {
-
-    checkAnswer(s"""select avg(contractNumber) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select avg(contractNumber) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_042")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_043
-  test("TABLE_DICTIONARY_INCLUDE_043", Include) {
-
-    checkAnswer(s"""select min(contractNumber) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select min(contractNumber) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_043")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_044
-  test("TABLE_DICTIONARY_INCLUDE_044", Include) {
-
-    sql(s"""select variance(contractNumber) as a   from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_045
-  ignore("TABLE_DICTIONARY_INCLUDE_045", Include) {
-
-    checkAnswer(s"""select var_pop(contractNumber) as a from (select * from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""",
-      s"""select var_pop(contractNumber) as a from (select * from TABLE_DICTIONARY_INCLUDE_hive order by contractNumber) t""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_045")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_046
-  test("TABLE_DICTIONARY_INCLUDE_046", Include) {
-
-    checkAnswer(s"""select var_samp(contractNumber) as a from  (select * from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""",
-      s"""select var_samp(contractNumber) as a from  (select * from TABLE_DICTIONARY_INCLUDE_hive order by contractNumber) t""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_046")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_047
-  test("TABLE_DICTIONARY_INCLUDE_047", Include) {
-
-    sql(s"""select stddev_pop(contractNumber) as a  from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_048
-  test("TABLE_DICTIONARY_INCLUDE_048", Include) {
-
-    sql(s"""select stddev_samp(contractNumber)  as a from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_049
-  test("TABLE_DICTIONARY_INCLUDE_049", Include) {
-
-    sql(s"""select covar_pop(contractNumber,contractNumber) as a  from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_050
-  test("TABLE_DICTIONARY_INCLUDE_050", Include) {
-
-    sql(s"""select covar_samp(contractNumber,contractNumber) as a  from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_051
-  test("TABLE_DICTIONARY_INCLUDE_051", Include) {
-
-    checkAnswer(s"""select corr(contractNumber,contractNumber)  as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select corr(contractNumber,contractNumber)  as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_051")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_052
-  test("TABLE_DICTIONARY_INCLUDE_052", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,0.2) as a  from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_053
-  test("TABLE_DICTIONARY_INCLUDE_053", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,0.2,5) as a  from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_054
-  test("TABLE_DICTIONARY_INCLUDE_054", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,array(0.2,0.3,0.99))  as a from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_055
-  test("TABLE_DICTIONARY_INCLUDE_055", Include) {
-
-    sql(s"""select percentile_approx(contractNumber,array(0.2,0.3,0.99),5) as a from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_056
-  test("TABLE_DICTIONARY_INCLUDE_056", Include) {
-
-    sql(s"""select histogram_numeric(contractNumber,2)  as a from (select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_057
-  test("TABLE_DICTIONARY_INCLUDE_057", Include) {
-
-    checkAnswer(s"""select contractNumber+ 10 as a  from TABLE_DICTIONARY_INCLUDE order by a""",
-      s"""select contractNumber+ 10 as a  from TABLE_DICTIONARY_INCLUDE_hive order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_057")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_058
-  test("TABLE_DICTIONARY_INCLUDE_058", Include) {
-
-    checkAnswer(s"""select min(contractNumber), max(contractNumber+ 10) Total from TABLE_DICTIONARY_INCLUDE group by  channelsId order by Total""",
-      s"""select min(contractNumber), max(contractNumber+ 10) Total from TABLE_DICTIONARY_INCLUDE_hive group by  channelsId order by Total""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_058")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_059
-  test("TABLE_DICTIONARY_INCLUDE_059", Include) {
-
-    sql(s"""select last(contractNumber) a from TABLE_DICTIONARY_INCLUDE  order by a""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_060
-  test("TABLE_DICTIONARY_INCLUDE_060", Include) {
-
-    checkAnswer(s"""select FIRST(contractNumber) a from TABLE_DICTIONARY_INCLUDE order by a""",
-      s"""select FIRST(contractNumber) a from TABLE_DICTIONARY_INCLUDE_hive order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_060")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_061
-  test("TABLE_DICTIONARY_INCLUDE_061", Include) {
-
-    checkAnswer(s"""select contractNumber,count(contractNumber) a from TABLE_DICTIONARY_INCLUDE group by contractNumber order by contractNumber""",
-      s"""select contractNumber,count(contractNumber) a from TABLE_DICTIONARY_INCLUDE_hive group by contractNumber order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_061")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_062
-  test("TABLE_DICTIONARY_INCLUDE_062", Include) {
-
-    checkAnswer(s"""select Lower(contractNumber) a  from TABLE_DICTIONARY_INCLUDE order by contractNumber""",
-      s"""select Lower(contractNumber) a  from TABLE_DICTIONARY_INCLUDE_hive order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_062")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_063
-  test("TABLE_DICTIONARY_INCLUDE_063", Include) {
-
-    checkAnswer(s"""select distinct contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber""",
-      s"""select distinct contractNumber from TABLE_DICTIONARY_INCLUDE_hive order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_063")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_064
-  test("TABLE_DICTIONARY_INCLUDE_064", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE order by contractNumber limit 101""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive order by contractNumber limit 101""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_064")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_065
-  test("TABLE_DICTIONARY_INCLUDE_065", Include) {
-
-    checkAnswer(s"""select contractNumber as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select contractNumber as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_065")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_066
-  test("TABLE_DICTIONARY_INCLUDE_066", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where  (contractNumber == 9223372047700) and (imei=='1AA100012')""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where  (contractNumber == 9223372047700) and (imei=='1AA100012')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_066")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_067
-  test("TABLE_DICTIONARY_INCLUDE_067", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber !=9223372047700 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber !=9223372047700 order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_067")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_068
-  test("TABLE_DICTIONARY_INCLUDE_068", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color') order by contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color') order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_068")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_069
-  test("TABLE_DICTIONARY_INCLUDE_069", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber !=9223372047700 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber !=9223372047700 order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_069")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_070
-  test("TABLE_DICTIONARY_INCLUDE_070", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber >9223372047700 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber >9223372047700 order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_070")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_071
-  test("TABLE_DICTIONARY_INCLUDE_071", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE where contractNumber<>contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE_hive where contractNumber<>contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_071")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_072
-  test("TABLE_DICTIONARY_INCLUDE_072", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber != Latest_areaId order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber != Latest_areaId order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_072")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_073
-  test("TABLE_DICTIONARY_INCLUDE_073", Include) {
-
-    checkAnswer(s"""select contractNumber, contractNumber from TABLE_DICTIONARY_INCLUDE where Latest_areaId<contractNumber order by contractNumber""",
-      s"""select contractNumber, contractNumber from TABLE_DICTIONARY_INCLUDE_hive where Latest_areaId<contractNumber order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_073")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_074
-  test("TABLE_DICTIONARY_INCLUDE_074", Include) {
-
-    checkAnswer(s"""select contractNumber, contractNumber from TABLE_DICTIONARY_INCLUDE where Latest_DAY<=contractNumber order by contractNumber""",
-      s"""select contractNumber, contractNumber from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY<=contractNumber order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_074")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_075
-  test("TABLE_DICTIONARY_INCLUDE_075", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber <1000 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber <1000 order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_075")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_076
-  test("TABLE_DICTIONARY_INCLUDE_076", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber >1000 order by contractNumber""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber >1000 order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_076")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_077
-  test("TABLE_DICTIONARY_INCLUDE_077", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE where contractNumber IS NULL order by contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE_hive where contractNumber IS NULL order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_077")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_078
-  test("TABLE_DICTIONARY_INCLUDE_078", Include) {
-
-    checkAnswer(s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE where Latest_DAY IS NOT NULL order by contractNumber""",
-      s"""select contractNumber  from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY IS NOT NULL order by contractNumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_078")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_079
-  test("TABLE_DICTIONARY_INCLUDE_079", Include) {
-
-    checkAnswer(s"""Select count(Latest_DAY) from TABLE_DICTIONARY_INCLUDE""",
-      s"""Select count(Latest_DAY) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_079")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_080
-  test("TABLE_DICTIONARY_INCLUDE_080", Include) {
-
-    checkAnswer(s"""select count(DISTINCT Latest_DAY) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT Latest_DAY) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_080")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_081
-  test("TABLE_DICTIONARY_INCLUDE_081", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)+10 as a ,Latest_DAY  from TABLE_DICTIONARY_INCLUDE group by Latest_DAY order by a""",
-      s"""select sum(Latest_DAY)+10 as a ,Latest_DAY  from TABLE_DICTIONARY_INCLUDE_hive group by Latest_DAY order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_081")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_082
-  test("TABLE_DICTIONARY_INCLUDE_082", Include) {
-
-    checkAnswer(s"""select max(Latest_DAY),min(Latest_DAY) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(Latest_DAY),min(Latest_DAY) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_082")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_083
-  test("TABLE_DICTIONARY_INCLUDE_083", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_083")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_084
-  test("TABLE_DICTIONARY_INCLUDE_084", Include) {
-
-    checkAnswer(s"""select avg(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select avg(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_084")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_085
-  test("TABLE_DICTIONARY_INCLUDE_085", Include) {
-
-    checkAnswer(s"""select min(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select min(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_085")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_086
-  test("TABLE_DICTIONARY_INCLUDE_086", Include) {
-
-    sql(s"""select variance(Latest_DAY) as a   from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_087
-  test("TABLE_DICTIONARY_INCLUDE_087", Include) {
-
-    sql(s"""select var_pop(Latest_DAY)  as a from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_088
-  test("TABLE_DICTIONARY_INCLUDE_088", Include) {
-
-    sql(s"""select var_samp(Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_089
-  test("TABLE_DICTIONARY_INCLUDE_089", Include) {
-
-    sql(s"""select stddev_pop(Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_090
-  test("TABLE_DICTIONARY_INCLUDE_090", Include) {
-
-    sql(s"""select stddev_samp(Latest_DAY)  as a from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_091
-  test("TABLE_DICTIONARY_INCLUDE_091", Include) {
-
-    sql(s"""select covar_pop(Latest_DAY,Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_092
-  test("TABLE_DICTIONARY_INCLUDE_092", Include) {
-
-    sql(s"""select covar_samp(Latest_DAY,Latest_DAY) as a  from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_093
-  test("TABLE_DICTIONARY_INCLUDE_093", Include) {
-
-    checkAnswer(s"""select corr(Latest_DAY,Latest_DAY)  as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select corr(Latest_DAY,Latest_DAY)  as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_093")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_094
-  test("TABLE_DICTIONARY_INCLUDE_094", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,0.2) as a  from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_095
-  test("TABLE_DICTIONARY_INCLUDE_095", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,0.2,5) as a  from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_096
-  test("TABLE_DICTIONARY_INCLUDE_096", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,array(0.2,0.3,0.99))  as a from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_097
-  test("TABLE_DICTIONARY_INCLUDE_097", Include) {
-
-    sql(s"""select percentile_approx(Latest_DAY,array(0.2,0.3,0.99),5) as a from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_098
-  test("TABLE_DICTIONARY_INCLUDE_098", Include) {
-
-    sql(s"""select histogram_numeric(Latest_DAY,2)  as a from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_099
-  test("TABLE_DICTIONARY_INCLUDE_099", Include) {
-
-    checkAnswer(s"""select Latest_DAY, Latest_DAY+ 10 as a  from TABLE_DICTIONARY_INCLUDE order by a""",
-      s"""select Latest_DAY, Latest_DAY+ 10 as a  from TABLE_DICTIONARY_INCLUDE_hive order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_099")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_100
-  test("TABLE_DICTIONARY_INCLUDE_100", Include) {
-
-    checkAnswer(s"""select min(Latest_DAY) a, max(Latest_DAY+ 10) Total from TABLE_DICTIONARY_INCLUDE group by  channelsId order by a,Total""",
-      s"""select min(Latest_DAY) a, max(Latest_DAY+ 10) Total from TABLE_DICTIONARY_INCLUDE_hive group by  channelsId order by a,Total""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_100")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_101
-  test("TABLE_DICTIONARY_INCLUDE_101", Include) {
-
-    sql(s"""select last(Latest_DAY) a from TABLE_DICTIONARY_INCLUDE order by a""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_102
-  test("TABLE_DICTIONARY_INCLUDE_102", Include) {
-
-    sql(s"""select FIRST(Latest_DAY) a from (select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_103
-  test("TABLE_DICTIONARY_INCLUDE_103", Include) {
-
-    checkAnswer(s"""select Latest_DAY,count(Latest_DAY) a from TABLE_DICTIONARY_INCLUDE group by Latest_DAY order by Latest_DAY""",
-      s"""select Latest_DAY,count(Latest_DAY) a from TABLE_DICTIONARY_INCLUDE_hive group by Latest_DAY order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_103")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_104
-  test("TABLE_DICTIONARY_INCLUDE_104", Include) {
-
-    checkAnswer(s"""select Lower(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE order by a""",
-      s"""select Lower(Latest_DAY) a  from TABLE_DICTIONARY_INCLUDE_hive order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_104")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_105
-  test("TABLE_DICTIONARY_INCLUDE_105", Include) {
-
-    checkAnswer(s"""select distinct Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY""",
-      s"""select distinct Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_105")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_106
-  test("TABLE_DICTIONARY_INCLUDE_106", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE order by Latest_DAY limit 101""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive order by Latest_DAY limit 101""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_106")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_107
-  test("TABLE_DICTIONARY_INCLUDE_107", Include) {
-
-    checkAnswer(s"""select Latest_DAY as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select Latest_DAY as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_107")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_108
-  test("TABLE_DICTIONARY_INCLUDE_108", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where  (Latest_DAY == 1234567890123450.0000000000)  and (imei=='1AA1')""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where  (Latest_DAY == 1234567890123450.0000000000)  and (imei=='1AA1')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_108")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_109
-  test("TABLE_DICTIONARY_INCLUDE_109", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_109")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_110
-  test("TABLE_DICTIONARY_INCLUDE_110", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_110")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_111
-  test("TABLE_DICTIONARY_INCLUDE_111", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY !=1234567890123450.0000000000  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_111")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_112
-  test("TABLE_DICTIONARY_INCLUDE_112", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY >1234567890123450.0000000000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY >1234567890123450.0000000000  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_112")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_113
-  test("TABLE_DICTIONARY_INCLUDE_113", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE where Latest_DAY<>Latest_DAY""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY<>Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_113")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_114
-  test("TABLE_DICTIONARY_INCLUDE_114", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY != Latest_areaId order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY != Latest_areaId order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_114")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_115
-  test("TABLE_DICTIONARY_INCLUDE_115", Include) {
-
-    checkAnswer(s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_areaId<Latest_DAY order by Latest_DAY""",
-      s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_areaId<Latest_DAY order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_115")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_116
-  test("TABLE_DICTIONARY_INCLUDE_116", Include) {
-
-    checkAnswer(s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY<=Latest_DAY  order by Latest_DAY""",
-      s"""select Latest_DAY, Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY<=Latest_DAY  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_116")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_117
-  test("TABLE_DICTIONARY_INCLUDE_117", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY <1000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY <1000  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_117")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_118
-  test("TABLE_DICTIONARY_INCLUDE_118", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY >1000  order by Latest_DAY""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY >1000  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_118")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_119
-  test("TABLE_DICTIONARY_INCLUDE_119", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE where Latest_DAY IS NULL  order by Latest_DAY""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY IS NULL  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_119")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_120
-  test("TABLE_DICTIONARY_INCLUDE_120", Include) {
-
-    checkAnswer(s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE where Latest_DAY IS NOT NULL  order by Latest_DAY""",
-      s"""select Latest_DAY  from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY IS NOT NULL  order by Latest_DAY""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_120")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_121
-  test("TABLE_DICTIONARY_INCLUDE_121", Include) {
-
-    checkAnswer(s"""Select count(gamePointId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""Select count(gamePointId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_121")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_122
-  test("TABLE_DICTIONARY_INCLUDE_122", Include) {
-
-    checkAnswer(s"""select count(DISTINCT gamePointId) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT gamePointId) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_122")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_123
-  test("TABLE_DICTIONARY_INCLUDE_123", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)+10 as a ,gamePointId  from TABLE_DICTIONARY_INCLUDE group by gamePointId order by a""",
-      s"""select sum(gamePointId)+10 as a ,gamePointId  from TABLE_DICTIONARY_INCLUDE_hive group by gamePointId order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_123")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_124
-  test("TABLE_DICTIONARY_INCLUDE_124", Include) {
-
-    checkAnswer(s"""select max(gamePointId),min(gamePointId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(gamePointId),min(gamePointId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_124")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_125
-  test("TABLE_DICTIONARY_INCLUDE_125", Include) {
-
-    checkAnswer(s"""select sum(gamePointId) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(gamePointId) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_125")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_126
-  test("TABLE_DICTIONARY_INCLUDE_126", Include) {
-
-    checkAnswer(s"""select avg(gamePointId) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select avg(gamePointId) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_126")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_127
-  test("TABLE_DICTIONARY_INCLUDE_127", Include) {
-
-    checkAnswer(s"""select min(gamePointId) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select min(gamePointId) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_127")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_128
-  test("TABLE_DICTIONARY_INCLUDE_128", Include) {
-
-    sql(s"""select variance(gamePointId) as a   from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_129
-  test("TABLE_DICTIONARY_INCLUDE_129", Include) {
-
-    sql(s"""select var_pop(gamePointId)  as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_130
-  test("TABLE_DICTIONARY_INCLUDE_130", Include) {
-
-    sql(s"""select var_samp(gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_131
-  test("TABLE_DICTIONARY_INCLUDE_131", Include) {
-
-    sql(s"""select stddev_pop(gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_132
-  test("TABLE_DICTIONARY_INCLUDE_132", Include) {
-
-    sql(s"""select stddev_samp(gamePointId)  as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_133
-  test("TABLE_DICTIONARY_INCLUDE_133", Include) {
-
-    sql(s"""select covar_pop(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_134
-  test("TABLE_DICTIONARY_INCLUDE_134", Include) {
-
-    sql(s"""select covar_samp(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_135
-  test("TABLE_DICTIONARY_INCLUDE_135", Include) {
-
-    checkAnswer(s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_135")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_136
-  test("TABLE_DICTIONARY_INCLUDE_136", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_137
-  test("TABLE_DICTIONARY_INCLUDE_137", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2,5) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_138
-  test("TABLE_DICTIONARY_INCLUDE_138", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99))  as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_139
-  test("TABLE_DICTIONARY_INCLUDE_139", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99),5) as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_140
-  test("TABLE_DICTIONARY_INCLUDE_140", Include) {
-
-    sql(s"""select histogram_numeric(gamePointId,2)  as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_141
-  test("TABLE_DICTIONARY_INCLUDE_141", Include) {
-
-    checkAnswer(s"""select gamePointId, gamePointId+ 10 as a  from TABLE_DICTIONARY_INCLUDE order by a""",
-      s"""select gamePointId, gamePointId+ 10 as a  from TABLE_DICTIONARY_INCLUDE_hive order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_141")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_142
-  test("TABLE_DICTIONARY_INCLUDE_142", Include) {
-
-    checkAnswer(s"""select min(gamePointId), max(gamePointId+ 10) Total from TABLE_DICTIONARY_INCLUDE group by  channelsId order by Total""",
-      s"""select min(gamePointId), max(gamePointId+ 10) Total from TABLE_DICTIONARY_INCLUDE_hive group by  channelsId order by Total""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_142")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_143
-  test("TABLE_DICTIONARY_INCLUDE_143", Include) {
-
-    sql(s"""select last(gamePointId) a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_144
-  test("TABLE_DICTIONARY_INCLUDE_144", Include) {
-
-    sql(s"""select FIRST(gamePointId) a from TABLE_DICTIONARY_INCLUDE order by a""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_145
-  test("TABLE_DICTIONARY_INCLUDE_145", Include) {
-
-    checkAnswer(s"""select gamePointId,count(gamePointId) a from TABLE_DICTIONARY_INCLUDE group by gamePointId order by gamePointId""",
-      s"""select gamePointId,count(gamePointId) a from TABLE_DICTIONARY_INCLUDE_hive group by gamePointId order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_145")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_146
-  test("TABLE_DICTIONARY_INCLUDE_146", Include) {
-
-    checkAnswer(s"""select Lower(gamePointId) a  from TABLE_DICTIONARY_INCLUDE order by gamePointId""",
-      s"""select Lower(gamePointId) a  from TABLE_DICTIONARY_INCLUDE_hive order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_146")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_147
-  test("TABLE_DICTIONARY_INCLUDE_147", Include) {
-
-    checkAnswer(s"""select distinct gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId""",
-      s"""select distinct gamePointId from TABLE_DICTIONARY_INCLUDE_hive order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_147")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_148
-  test("TABLE_DICTIONARY_INCLUDE_148", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE  order by gamePointId limit 101""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive  order by gamePointId limit 101""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_148")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_149
-  test("TABLE_DICTIONARY_INCLUDE_149", Include) {
-
-    checkAnswer(s"""select gamePointId as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select gamePointId as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_149")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_150
-  test("TABLE_DICTIONARY_INCLUDE_150", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where  (gamePointId == 4.70133553923674E43) and (imei=='1AA100084')""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where  (gamePointId == 4.70133553923674E43) and (imei=='1AA100084')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_150")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_151
-  test("TABLE_DICTIONARY_INCLUDE_151", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId !=4.70133553923674E43  order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId !=4.70133553923674E43  order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_151")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_152
-  test("TABLE_DICTIONARY_INCLUDE_152", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_152")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_153
-  test("TABLE_DICTIONARY_INCLUDE_153", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId !=4.70133553923674E43""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId !=4.70133553923674E43""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_153")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_154
-  test("TABLE_DICTIONARY_INCLUDE_154", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId >4.70133553923674E43""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId >4.70133553923674E43""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_154")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_155
-  test("TABLE_DICTIONARY_INCLUDE_155", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE where gamePointId<>gamePointId""",
-      s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE_hive where gamePointId<>gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_155")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_156
-  test("TABLE_DICTIONARY_INCLUDE_156", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId != Latest_areaId  order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId != Latest_areaId  order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_156")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_157
-  test("TABLE_DICTIONARY_INCLUDE_157", Include) {
-
-    checkAnswer(s"""select gamePointId, gamePointId from TABLE_DICTIONARY_INCLUDE where Latest_areaId<gamePointId  order by gamePointId""",
-      s"""select gamePointId, gamePointId from TABLE_DICTIONARY_INCLUDE_hive where Latest_areaId<gamePointId  order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_157")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_158
-  test("TABLE_DICTIONARY_INCLUDE_158", Include) {
-
-    checkAnswer(s"""select gamePointId, gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId<=gamePointId  order by gamePointId""",
-      s"""select gamePointId, gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId<=gamePointId  order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_158")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_159
-  test("TABLE_DICTIONARY_INCLUDE_159", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId <1000 order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId <1000 order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_159")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_160
-  test("TABLE_DICTIONARY_INCLUDE_160", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId >1000 order by gamePointId""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId >1000 order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_160")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_161
-  test("TABLE_DICTIONARY_INCLUDE_161", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE where gamePointId IS NULL order by gamePointId""",
-      s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE_hive where gamePointId IS NULL order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_161")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_162
-  test("TABLE_DICTIONARY_INCLUDE_162", Include) {
-
-    checkAnswer(s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE where gamePointId IS NOT NULL order by gamePointId""",
-      s"""select gamePointId  from TABLE_DICTIONARY_INCLUDE_hive where gamePointId IS NOT NULL order by gamePointId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_162")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_163
-  test("TABLE_DICTIONARY_INCLUDE_163", Include) {
-
-    checkAnswer(s"""Select count(productionDate) from TABLE_DICTIONARY_INCLUDE""",
-      s"""Select count(productionDate) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_163")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_164
-  test("TABLE_DICTIONARY_INCLUDE_164", Include) {
-
-    checkAnswer(s"""select count(DISTINCT productionDate) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT productionDate) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_164")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_165
-  test("TABLE_DICTIONARY_INCLUDE_165", Include) {
-
-    checkAnswer(s"""select sum(productionDate)+10 as a ,productionDate  from TABLE_DICTIONARY_INCLUDE group by productionDate order by productionDate""",
-      s"""select sum(productionDate)+10 as a ,productionDate  from TABLE_DICTIONARY_INCLUDE_hive group by productionDate order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_165")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_166
-  test("TABLE_DICTIONARY_INCLUDE_166", Include) {
-
-    checkAnswer(s"""select max(productionDate),min(productionDate) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(productionDate),min(productionDate) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_166")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_167
-  test("TABLE_DICTIONARY_INCLUDE_167", Include) {
-
-    checkAnswer(s"""select sum(productionDate) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(productionDate) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_167")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_168
-  test("TABLE_DICTIONARY_INCLUDE_168", Include) {
-
-    checkAnswer(s"""select avg(productionDate) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select avg(productionDate) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_168")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_169
-  test("TABLE_DICTIONARY_INCLUDE_169", Include) {
-
-    checkAnswer(s"""select min(productionDate) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select min(productionDate) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_169")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_170
-  test("TABLE_DICTIONARY_INCLUDE_170", Include) {
-
-    sql(s"""select variance(gamePointId) as a   from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_171
-  ignore("TABLE_DICTIONARY_INCLUDE_171", Include) {
-
-    checkAnswer(s"""select var_pop(gamePointId) as a from (select * from TABLE_DICTIONARY_INCLUDE order by gamePointId)""",
-      s"""select var_pop(gamePointId) as a from (select * from TABLE_DICTIONARY_INCLUDE_hive order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_171")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_172
-  ignore("TABLE_DICTIONARY_INCLUDE_172", Include) {
-
-    checkAnswer(s"""select var_samp(gamePointId) as a from (select * from TABLE_DICTIONARY_INCLUDE order by gamePointId)""",
-      s"""select var_samp(gamePointId) as a from (select * from TABLE_DICTIONARY_INCLUDE_hive order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_172")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_173
-  test("TABLE_DICTIONARY_INCLUDE_173", Include) {
-
-    sql(s"""select stddev_pop(gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_174
-  test("TABLE_DICTIONARY_INCLUDE_174", Include) {
-
-    sql(s"""select stddev_samp(gamePointId)  as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_175
-  test("TABLE_DICTIONARY_INCLUDE_175", Include) {
-
-    sql(s"""select covar_pop(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_176
-  test("TABLE_DICTIONARY_INCLUDE_176", Include) {
-
-    sql(s"""select covar_samp(gamePointId,gamePointId) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_177
-  test("TABLE_DICTIONARY_INCLUDE_177", Include) {
-
-    checkAnswer(s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select corr(gamePointId,gamePointId)  as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_177")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_178
-  test("TABLE_DICTIONARY_INCLUDE_178", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_179
-  test("TABLE_DICTIONARY_INCLUDE_179", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,0.2,5) as a  from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_180
-  test("TABLE_DICTIONARY_INCLUDE_180", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99))  as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_181
-  test("TABLE_DICTIONARY_INCLUDE_181", Include) {
-
-    sql(s"""select percentile_approx(gamePointId,array(0.2,0.3,0.99),5) as a from (select gamePointId from TABLE_DICTIONARY_INCLUDE order by gamePointId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_182
-  test("TABLE_DICTIONARY_INCLUDE_182", Include) {
-
-    sql(s"""select histogram_numeric(productionDate,2)  as a from (select productionDate from TABLE_DICTIONARY_INCLUDE order by productionDate) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_183
-  ignore("TABLE_DICTIONARY_INCLUDE_183", Include) {
-
-    checkAnswer(s"""select last(productionDate) a from TABLE_DICTIONARY_INCLUDE order by a""",
-      s"""select last(productionDate) a from TABLE_DICTIONARY_INCLUDE_hive order by a""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_183")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_184
-  test("TABLE_DICTIONARY_INCLUDE_184", Include) {
-
-    sql(s"""select FIRST(productionDate) a from TABLE_DICTIONARY_INCLUDE  order by a""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_185
-  test("TABLE_DICTIONARY_INCLUDE_185", Include) {
-
-    checkAnswer(s"""select productionDate,count(productionDate) a from TABLE_DICTIONARY_INCLUDE group by productionDate order by productionDate""",
-      s"""select productionDate,count(productionDate) a from TABLE_DICTIONARY_INCLUDE_hive group by productionDate order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_185")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_186
-  test("TABLE_DICTIONARY_INCLUDE_186", Include) {
-
-    checkAnswer(s"""select Lower(productionDate) a  from TABLE_DICTIONARY_INCLUDE order by productionDate""",
-      s"""select Lower(productionDate) a  from TABLE_DICTIONARY_INCLUDE_hive order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_186")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_187
-  test("TABLE_DICTIONARY_INCLUDE_187", Include) {
-
-    checkAnswer(s"""select distinct productionDate from TABLE_DICTIONARY_INCLUDE order by productionDate""",
-      s"""select distinct productionDate from TABLE_DICTIONARY_INCLUDE_hive order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_187")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_188
-  test("TABLE_DICTIONARY_INCLUDE_188", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE order by productionDate limit 101""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive order by productionDate limit 101""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_188")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_189
-  test("TABLE_DICTIONARY_INCLUDE_189", Include) {
-
-    checkAnswer(s"""select productionDate as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select productionDate as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_189")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_190
-  test("TABLE_DICTIONARY_INCLUDE_190", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where  (productionDate == '2015-07-01 12:07:28.0') and (productionDate=='2015-07-01 12:07:28.0')""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where  (productionDate == '2015-07-01 12:07:28.0') and (productionDate=='2015-07-01 12:07:28.0')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_190")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_191
-  test("TABLE_DICTIONARY_INCLUDE_191", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate !='2015-07-01 12:07:28.0' order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate !='2015-07-01 12:07:28.0' order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_191")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_192
-  test("TABLE_DICTIONARY_INCLUDE_192", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select productionDate  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_192")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_193
-  test("TABLE_DICTIONARY_INCLUDE_193", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate !='2015-07-01 12:07:28.0' order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate !='2015-07-01 12:07:28.0' order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_193")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_194
-  test("TABLE_DICTIONARY_INCLUDE_194", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate >'2015-07-01 12:07:28.0' order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate >'2015-07-01 12:07:28.0' order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_194")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_195
-  test("TABLE_DICTIONARY_INCLUDE_195", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_INCLUDE where productionDate<>productionDate order by productionDate""",
-      s"""select productionDate  from TABLE_DICTIONARY_INCLUDE_hive where productionDate<>productionDate order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_195")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_196
-  test("TABLE_DICTIONARY_INCLUDE_196", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate != Latest_areaId order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate != Latest_areaId order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_196")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_197
-  test("TABLE_DICTIONARY_INCLUDE_197", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where Latest_areaId<productionDate order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where Latest_areaId<productionDate order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_197")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_198
-  test("TABLE_DICTIONARY_INCLUDE_198", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate<=productionDate order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate<=productionDate order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_198")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_199
-  test("TABLE_DICTIONARY_INCLUDE_199", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate <cast('2015-07-01 12:07:28.0' as timestamp) order by productionDate""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate <cast('2015-07-01 12:07:28.0' as timestamp) order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_199")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_200
-  test("TABLE_DICTIONARY_INCLUDE_200", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_INCLUDE where productionDate IS NULL""",
-      s"""select productionDate  from TABLE_DICTIONARY_INCLUDE_hive where productionDate IS NULL""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_200")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_201
-  test("TABLE_DICTIONARY_INCLUDE_201", Include) {
-
-    checkAnswer(s"""select productionDate  from TABLE_DICTIONARY_INCLUDE where productionDate IS NOT NULL order by productionDate""",
-      s"""select productionDate  from TABLE_DICTIONARY_INCLUDE_hive where productionDate IS NOT NULL order by productionDate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_201")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_202
-  test("TABLE_DICTIONARY_INCLUDE_202", Include) {
-
-    checkAnswer(s"""Select count(deviceInformationId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""Select count(deviceInformationId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_202")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_203
-  test("TABLE_DICTIONARY_INCLUDE_203", Include) {
-
-    checkAnswer(s"""select count(DISTINCT deviceInformationId) as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(DISTINCT deviceInformationId) as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_203")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_204
-  test("TABLE_DICTIONARY_INCLUDE_204", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)+10 as a ,deviceInformationId  from TABLE_DICTIONARY_INCLUDE group by deviceInformationId order by deviceInformationId""",
-      s"""select sum(deviceInformationId)+10 as a ,deviceInformationId  from TABLE_DICTIONARY_INCLUDE_hive group by deviceInformationId order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_204")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_205
-  test("TABLE_DICTIONARY_INCLUDE_205", Include) {
-
-    checkAnswer(s"""select max(deviceInformationId),min(deviceInformationId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select max(deviceInformationId),min(deviceInformationId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_205")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_206
-  test("TABLE_DICTIONARY_INCLUDE_206", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_206")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_207
-  test("TABLE_DICTIONARY_INCLUDE_207", Include) {
-
-    checkAnswer(s"""select avg(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select avg(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_207")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_208
-  test("TABLE_DICTIONARY_INCLUDE_208", Include) {
-
-    checkAnswer(s"""select min(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select min(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_208")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_209
-  test("TABLE_DICTIONARY_INCLUDE_209", Include) {
-
-    sql(s"""select variance(deviceInformationId) as a   from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_210
-  ignore("TABLE_DICTIONARY_INCLUDE_210", Include) {
-
-    checkAnswer(s"""select var_pop(deviceInformationId)  as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select var_pop(deviceInformationId)  as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_210")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_211
-  ignore("TABLE_DICTIONARY_INCLUDE_211", Include) {
-
-    checkAnswer(s"""select var_samp(deviceInformationId) as a  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select var_samp(deviceInformationId) as a  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_211")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_212
-  test("TABLE_DICTIONARY_INCLUDE_212", Include) {
-
-    sql(s"""select stddev_pop(deviceInformationId) as a  from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_213
-  test("TABLE_DICTIONARY_INCLUDE_213", Include) {
-
-    sql(s"""select stddev_samp(deviceInformationId)  as a from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_214
-  test("TABLE_DICTIONARY_INCLUDE_214", Include) {
-
-    sql(s"""select covar_pop(deviceInformationId,deviceInformationId) as a  from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_215
-  test("TABLE_DICTIONARY_INCLUDE_215", Include) {
-
-    sql(s"""select covar_samp(deviceInformationId,deviceInformationId) as a  from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_216
-  test("TABLE_DICTIONARY_INCLUDE_216", Include) {
-
-    checkAnswer(s"""select corr(deviceInformationId,deviceInformationId)  as a from TABLE_DICTIONARY_INCLUDE""",
-      s"""select corr(deviceInformationId,deviceInformationId)  as a from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_216")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_217
-  test("TABLE_DICTIONARY_INCLUDE_217", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,0.2) as a  from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_218
-  test("TABLE_DICTIONARY_INCLUDE_218", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,0.2,5) as a  from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_219
-  test("TABLE_DICTIONARY_INCLUDE_219", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,array(0.2,0.3,0.99))  as a from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_220
-  test("TABLE_DICTIONARY_INCLUDE_220", Include) {
-
-    sql(s"""select percentile_approx(deviceInformationId,array(0.2,0.3,0.99),5) as a from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_221
-  test("TABLE_DICTIONARY_INCLUDE_221", Include) {
-
-    sql(s"""select histogram_numeric(deviceInformationId,2)  as a from (select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId) t""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_222
-  test("TABLE_DICTIONARY_INCLUDE_222", Include) {
-
-    checkAnswer(s"""select deviceInformationId, deviceInformationId+ 10 as a  from TABLE_DICTIONARY_INCLUDE order by deviceInformationId""",
-      s"""select deviceInformationId, deviceInformationId+ 10 as a  from TABLE_DICTIONARY_INCLUDE_hive order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_222")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_223
-  test("TABLE_DICTIONARY_INCLUDE_223", Include) {
-
-    checkAnswer(s"""select min(deviceInformationId), max(deviceInformationId+ 10) Total from TABLE_DICTIONARY_INCLUDE group by  channelsId order by Total""",
-      s"""select min(deviceInformationId), max(deviceInformationId+ 10) Total from TABLE_DICTIONARY_INCLUDE_hive group by  channelsId order by Total""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_223")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_224
-  test("TABLE_DICTIONARY_INCLUDE_224", Include) {
-
-    sql(s"""select last(deviceInformationId) a from TABLE_DICTIONARY_INCLUDE order by a""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_225
-  test("TABLE_DICTIONARY_INCLUDE_225", Include) {
-
-    sql(s"""select FIRST(deviceInformationId) a from TABLE_DICTIONARY_INCLUDE order by a""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_226
-  test("TABLE_DICTIONARY_INCLUDE_226", Include) {
-
-    checkAnswer(s"""select deviceInformationId,count(deviceInformationId) a from TABLE_DICTIONARY_INCLUDE group by deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId,count(deviceInformationId) a from TABLE_DICTIONARY_INCLUDE_hive group by deviceInformationId order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_226")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_227
-  test("TABLE_DICTIONARY_INCLUDE_227", Include) {
-
-    checkAnswer(s"""select Lower(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE order by deviceInformationId""",
-      s"""select Lower(deviceInformationId) a  from TABLE_DICTIONARY_INCLUDE_hive order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_227")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_228
-  test("TABLE_DICTIONARY_INCLUDE_228", Include) {
-
-    checkAnswer(s"""select distinct deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId""",
-      s"""select distinct deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_228")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_229
-  test("TABLE_DICTIONARY_INCLUDE_229", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE order by deviceInformationId limit 101""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive order by deviceInformationId limit 101""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_229")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_230
-  test("TABLE_DICTIONARY_INCLUDE_230", Include) {
-
-    checkAnswer(s"""select deviceInformationId as a from TABLE_DICTIONARY_INCLUDE  order by a asc limit 10""",
-      s"""select deviceInformationId as a from TABLE_DICTIONARY_INCLUDE_hive  order by a asc limit 10""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_230")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_231
-  test("TABLE_DICTIONARY_INCLUDE_231", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where  (deviceInformationId == 100084) and (deviceInformationId==100084)""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where  (deviceInformationId == 100084) and (deviceInformationId==100084)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_231")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_232
-  test("TABLE_DICTIONARY_INCLUDE_232", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId !='100084' order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId !='100084' order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_232")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_233
-  test("TABLE_DICTIONARY_INCLUDE_233", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE_hive where (deviceInformationId=100 and deviceColor='1Device Color') OR (deviceInformationId=10 and deviceColor='0Device Color')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_233")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_234
-  test("TABLE_DICTIONARY_INCLUDE_234", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId !=100084 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId !=100084 order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_234")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_235
-  test("TABLE_DICTIONARY_INCLUDE_235", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId >100084 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId >100084 order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_235")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_236
-  test("TABLE_DICTIONARY_INCLUDE_236", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE where deviceInformationId<>deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId<>deviceInformationId order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_236")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_237
-  test("TABLE_DICTIONARY_INCLUDE_237", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId != Latest_areaId order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId != Latest_areaId order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_237")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_238
-  test("TABLE_DICTIONARY_INCLUDE_238", Include) {
-
-    checkAnswer(s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_INCLUDE where Latest_areaId<deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where Latest_areaId<deviceInformationId order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_238")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_239
-  test("TABLE_DICTIONARY_INCLUDE_239", Include) {
-
-    checkAnswer(s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId<=deviceInformationId order by deviceInformationId""",
-      s"""select deviceInformationId, deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId<=deviceInformationId order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_239")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_240
-  test("TABLE_DICTIONARY_INCLUDE_240", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId <1000 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId <1000 order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_240")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_241
-  test("TABLE_DICTIONARY_INCLUDE_241", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId >1000 order by deviceInformationId""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId >1000 order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_241")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_242
-  test("TABLE_DICTIONARY_INCLUDE_242", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE where deviceInformationId IS NULL order by deviceInformationId""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId IS NULL order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_242")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_243
-  test("TABLE_DICTIONARY_INCLUDE_243", Include) {
-
-    checkAnswer(s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE where deviceInformationId IS NOT NULL order by deviceInformationId""",
-      s"""select deviceInformationId  from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId IS NOT NULL order by deviceInformationId""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_243")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_244
-  test("TABLE_DICTIONARY_INCLUDE_244", Include) {
-
-    checkAnswer(s"""select sum(imei)+10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(imei)+10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_244")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_245
-  test("TABLE_DICTIONARY_INCLUDE_245", Include) {
-
-    checkAnswer(s"""select sum(imei)*10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(imei)*10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_245")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_246
-  test("TABLE_DICTIONARY_INCLUDE_246", Include) {
-
-    checkAnswer(s"""select sum(imei)/10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(imei)/10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_246")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_247
-  test("TABLE_DICTIONARY_INCLUDE_247", Include) {
-
-    checkAnswer(s"""select sum(imei)-10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(imei)-10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_247")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_248
-  test("TABLE_DICTIONARY_INCLUDE_248", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)+10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(contractNumber)+10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_248")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_249
-  test("TABLE_DICTIONARY_INCLUDE_249", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)*10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(contractNumber)*10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_249")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_250
-  test("TABLE_DICTIONARY_INCLUDE_250", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)/10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(contractNumber)/10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_250")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_251
-  test("TABLE_DICTIONARY_INCLUDE_251", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)-10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(contractNumber)-10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_251")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_252
-  test("TABLE_DICTIONARY_INCLUDE_252", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)+10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(Latest_DAY)+10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_252")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_253
-  test("TABLE_DICTIONARY_INCLUDE_253", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)*10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(Latest_DAY)*10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_253")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_254
-  test("TABLE_DICTIONARY_INCLUDE_254", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)/10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(Latest_DAY)/10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_254")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_255
-  test("TABLE_DICTIONARY_INCLUDE_255", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)-10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(Latest_DAY)-10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_255")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_256
-  test("TABLE_DICTIONARY_INCLUDE_256", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)+10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(gamePointId)+10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_256")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_257
-  test("TABLE_DICTIONARY_INCLUDE_257", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)*10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(gamePointId)*10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_257")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_258
-  test("TABLE_DICTIONARY_INCLUDE_258", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)/10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(gamePointId)/10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_258")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_259
-  test("TABLE_DICTIONARY_INCLUDE_259", Include) {
-
-    checkAnswer(s"""select sum(gamePointId)-10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(gamePointId)-10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_259")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_260
-  test("TABLE_DICTIONARY_INCLUDE_260", Include) {
-
-    checkAnswer(s"""select sum(productionDate)+10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(productionDate)+10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_260")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_261
-  test("TABLE_DICTIONARY_INCLUDE_261", Include) {
-
-    checkAnswer(s"""select sum(productionDate)*10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(productionDate)*10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_261")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_262
-  test("TABLE_DICTIONARY_INCLUDE_262", Include) {
-
-    checkAnswer(s"""select sum(productionDate)/10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(productionDate)/10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_262")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_263
-  test("TABLE_DICTIONARY_INCLUDE_263", Include) {
-
-    checkAnswer(s"""select sum(productionDate)-10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(productionDate)-10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_263")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_264
-  test("TABLE_DICTIONARY_INCLUDE_264", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)+10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(deviceInformationId)+10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_264")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_265
-  test("TABLE_DICTIONARY_INCLUDE_265", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)*10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(deviceInformationId)*10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_265")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_266
-  test("TABLE_DICTIONARY_INCLUDE_266", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)/10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(deviceInformationId)/10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_266")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_267
-  test("TABLE_DICTIONARY_INCLUDE_267", Include) {
-
-    checkAnswer(s"""select sum(deviceInformationId)-10 as a   from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(deviceInformationId)-10 as a   from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_267")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_292
-  test("TABLE_DICTIONARY_INCLUDE_292", Include) {
-
-    checkAnswer(s"""SELECT productiondate from TABLE_DICTIONARY_INCLUDE where productiondate LIKE '2015-09-30%'""",
-      s"""SELECT productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate LIKE '2015-09-30%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_292")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_293
-  test("TABLE_DICTIONARY_INCLUDE_293", Include) {
-
-    checkAnswer(s"""SELECT productiondate from TABLE_DICTIONARY_INCLUDE where productiondate LIKE '% %'""",
-      s"""SELECT productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate LIKE '% %'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_293")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_294
-  test("TABLE_DICTIONARY_INCLUDE_294", Include) {
-
-    checkAnswer(s"""SELECT productiondate from TABLE_DICTIONARY_INCLUDE where productiondate LIKE '%12:07:28'""",
-      s"""SELECT productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate LIKE '%12:07:28'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_294")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_295
-  test("TABLE_DICTIONARY_INCLUDE_295", Include) {
-
-    checkAnswer(s"""select contractnumber from TABLE_DICTIONARY_INCLUDE where contractnumber like '922337204%' """,
-      s"""select contractnumber from TABLE_DICTIONARY_INCLUDE_hive where contractnumber like '922337204%' """, "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_295")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_296
-  test("TABLE_DICTIONARY_INCLUDE_296", Include) {
-
-    checkAnswer(s"""select contractnumber from TABLE_DICTIONARY_INCLUDE where contractnumber like '%047800'""",
-      s"""select contractnumber from TABLE_DICTIONARY_INCLUDE_hive where contractnumber like '%047800'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_296")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_297
-  test("TABLE_DICTIONARY_INCLUDE_297", Include) {
-
-    checkAnswer(s"""select contractnumber from TABLE_DICTIONARY_INCLUDE where contractnumber like '%720%'""",
-      s"""select contractnumber from TABLE_DICTIONARY_INCLUDE_hive where contractnumber like '%720%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_297")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_298
-  test("TABLE_DICTIONARY_INCLUDE_298", Include) {
-
-    checkAnswer(s"""SELECT Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY like '12345678%'""",
-      s"""SELECT Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY like '12345678%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_298")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_299
-  test("TABLE_DICTIONARY_INCLUDE_299", Include) {
-
-    checkAnswer(s"""SELECT Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY like '%5678%'""",
-      s"""SELECT Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY like '%5678%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_299")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_300
-  test("TABLE_DICTIONARY_INCLUDE_300", Include) {
-
-    checkAnswer(s"""SELECT Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY like '1234567%'""",
-      s"""SELECT Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY like '1234567%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_300")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_301
-  test("TABLE_DICTIONARY_INCLUDE_301", Include) {
-
-    checkAnswer(s"""SELECT gamepointID from TABLE_DICTIONARY_INCLUDE where gamepointID like '1.1098347722%'""",
-      s"""SELECT gamepointID from TABLE_DICTIONARY_INCLUDE_hive where gamepointID like '1.1098347722%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_301")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_302
-  test("TABLE_DICTIONARY_INCLUDE_302", Include) {
-
-    checkAnswer(s"""SELECT gamepointID from TABLE_DICTIONARY_INCLUDE where gamepointID like '%8347722%'""",
-      s"""SELECT gamepointID from TABLE_DICTIONARY_INCLUDE_hive where gamepointID like '%8347722%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_302")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_303
-  test("TABLE_DICTIONARY_INCLUDE_303", Include) {
-
-    checkAnswer(s"""SELECT gamepointID from TABLE_DICTIONARY_INCLUDE where gamepointID like '%7512E42'""",
-      s"""SELECT gamepointID from TABLE_DICTIONARY_INCLUDE_hive where gamepointID like '%7512E42'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_303")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_304
-  test("TABLE_DICTIONARY_INCLUDE_304", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid like '1000%'""",
-      s"""SELECT deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid like '1000%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_304")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_305
-  test("TABLE_DICTIONARY_INCLUDE_305", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid like '%00%'""",
-      s"""SELECT deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid like '%00%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_305")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_306
-  test("TABLE_DICTIONARY_INCLUDE_306", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid like '%0084'""",
-      s"""SELECT deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid like '%0084'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_306")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_307
-  test("TABLE_DICTIONARY_INCLUDE_307", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei like '1AA10%'""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei like '1AA10%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_307")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_308
-  test("TABLE_DICTIONARY_INCLUDE_308", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei like '%A10%'""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei like '%A10%'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_308")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_309
-  test("TABLE_DICTIONARY_INCLUDE_309", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei like '%00084'""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei like '%00084'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_309")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_310
-  test("TABLE_DICTIONARY_INCLUDE_310", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei in ('1AA100074','1AA100075','1AA100077')""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei in ('1AA100074','1AA100075','1AA100077')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_310")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_311
-  test("TABLE_DICTIONARY_INCLUDE_311", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei not in ('1AA100074','1AA100075','1AA100077')""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei not in ('1AA100074','1AA100075','1AA100077')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_311")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_312
-  test("TABLE_DICTIONARY_INCLUDE_312", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid in (100081,100078,10008)""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid in (100081,100078,10008)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_312")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_313
-  test("TABLE_DICTIONARY_INCLUDE_313", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid not in (100081,100078,10008)""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid not in (100081,100078,10008)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_313")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_314
-  test("TABLE_DICTIONARY_INCLUDE_314", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_INCLUDE where productiondate in ('2015-10-04 12:07:28','2015-10-07%','2015-10-07 12:07:28')""",
-      s"""select productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate in ('2015-10-04 12:07:28','2015-10-07%','2015-10-07 12:07:28')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_314")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_315
-  test("TABLE_DICTIONARY_INCLUDE_315", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_INCLUDE where productiondate not in (cast('2015-10-04 12:07:28' as timestamp),cast('2015-10-07 12:07:28' as timestamp))""",
-      s"""select productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate not in (cast('2015-10-04 12:07:28' as timestamp),cast('2015-10-07 12:07:28' as timestamp))""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_315")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_316
-  test("TABLE_DICTIONARY_INCLUDE_316", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_INCLUDE where gamepointid in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""",
-      s"""select gamepointid from TABLE_DICTIONARY_INCLUDE_hive where gamepointid in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_316")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_317
-  test("TABLE_DICTIONARY_INCLUDE_317", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_INCLUDE where gamepointid not in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""",
-      s"""select gamepointid from TABLE_DICTIONARY_INCLUDE_hive where gamepointid not in (5.02870412391492E39,3.82247669125491E41,6.8591561117512E42)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_317")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_318
-  test("TABLE_DICTIONARY_INCLUDE_318", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY in (1234567890123530.0000000000,1234567890123520.0000000000)""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY in (1234567890123530.0000000000,1234567890123520.0000000000)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_318")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_319
-  test("TABLE_DICTIONARY_INCLUDE_319", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY not in (1234567890123530.0000000000,1234567890123520.0000000000)""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY not in (1234567890123530.0000000000,1234567890123520.0000000000)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_319")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_322
-  test("TABLE_DICTIONARY_INCLUDE_322", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei !='1AA100077'""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei !='1AA100077'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_322")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_323
-  test("TABLE_DICTIONARY_INCLUDE_323", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei NOT LIKE '1AA100077'""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei NOT LIKE '1AA100077'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_323")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_324
-  test("TABLE_DICTIONARY_INCLUDE_324", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid !=100078""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid !=100078""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_324")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_325
-  test("TABLE_DICTIONARY_INCLUDE_325", Include) {
-
-    checkAnswer(s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE where deviceinformationid NOT LIKE 100079""",
-      s"""select deviceinformationid from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationid NOT LIKE 100079""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_325")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_326
-  test("TABLE_DICTIONARY_INCLUDE_326", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_INCLUDE where productiondate !='2015-10-07 12:07:28'""",
-      s"""select productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate !='2015-10-07 12:07:28'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_326")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_327
-  ignore("TABLE_DICTIONARY_INCLUDE_327", Include) {
-
-    checkAnswer(s"""select productiondate from TABLE_DICTIONARY_INCLUDE where productiondate NOT LIKE '2015-10-07 12:07:28'""",
-      s"""select productiondate from TABLE_DICTIONARY_INCLUDE_hive where productiondate NOT LIKE '2015-10-07 12:07:28'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_327")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_328
-  test("TABLE_DICTIONARY_INCLUDE_328", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_INCLUDE where gamepointid !=6.8591561117512E42""",
-      s"""select gamepointid from TABLE_DICTIONARY_INCLUDE_hive where gamepointid !=6.8591561117512E42""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_328")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_329
-  test("TABLE_DICTIONARY_INCLUDE_329", Include) {
-
-    checkAnswer(s"""select gamepointid from TABLE_DICTIONARY_INCLUDE where gamepointid NOT LIKE 6.8591561117512E43""",
-      s"""select gamepointid from TABLE_DICTIONARY_INCLUDE_hive where gamepointid NOT LIKE 6.8591561117512E43""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_329")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_330
-  test("TABLE_DICTIONARY_INCLUDE_330", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY != 1234567890123520.0000000000""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY != 1234567890123520.0000000000""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_330")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_331
-  test("TABLE_DICTIONARY_INCLUDE_331", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY NOT LIKE 1234567890123520.0000000000""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY NOT LIKE 1234567890123520.0000000000""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_331")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_335
-  test("TABLE_DICTIONARY_INCLUDE_335", Include) {
-
-    checkAnswer(s"""SELECT productiondate,IMEI from TABLE_DICTIONARY_INCLUDE where IMEI RLIKE '1AA100077'""",
-      s"""SELECT productiondate,IMEI from TABLE_DICTIONARY_INCLUDE_hive where IMEI RLIKE '1AA100077'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_335")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_336
-  test("TABLE_DICTIONARY_INCLUDE_336", Include) {
-
-    checkAnswer(s"""SELECT deviceinformationId from TABLE_DICTIONARY_INCLUDE where deviceinformationId RLIKE '100079'""",
-      s"""SELECT deviceinformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceinformationId RLIKE '100079'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_336")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_337
-  test("TABLE_DICTIONARY_INCLUDE_337", Include) {
-
-    checkAnswer(s"""SELECT gamepointid from TABLE_DICTIONARY_INCLUDE where gamepointid RLIKE '1.61922711065643E42'""",
-      s"""SELECT gamepointid from TABLE_DICTIONARY_INCLUDE_hive where gamepointid RLIKE '1.61922711065643E42'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_337")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_338
-  test("TABLE_DICTIONARY_INCLUDE_338", Include) {
-
-    checkAnswer(s"""SELECT Latest_Day from TABLE_DICTIONARY_INCLUDE where Latest_Day RLIKE '1234567890123550.0000000000'""",
-      s"""SELECT Latest_Day from TABLE_DICTIONARY_INCLUDE_hive where Latest_Day RLIKE '1234567890123550.0000000000'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_338")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_339
-  test("TABLE_DICTIONARY_INCLUDE_339", Include) {
-
-    checkAnswer(s"""SELECT contractnumber from TABLE_DICTIONARY_INCLUDE where contractnumber RLIKE '9223372047800'""",
-      s"""SELECT contractnumber from TABLE_DICTIONARY_INCLUDE_hive where contractnumber RLIKE '9223372047800'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_339")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_340
-  test("TABLE_DICTIONARY_INCLUDE_340", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE a join table_dictionary_include b on a.productiondate=b.productiondate""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE_hive a join table_dictionary_include_hive b on a.productiondate=b.productiondate""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_340")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_341
-  test("TABLE_DICTIONARY_INCLUDE_341", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE a join table_dictionary_include b on a.deviceinformationid=b.deviceinformationid""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE_hive a join table_dictionary_include_hive b on a.deviceinformationid=b.deviceinformationid""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_341")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_342
-  test("TABLE_DICTIONARY_INCLUDE_342", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE a join table_dictionary_include b on a.imei=b.imei""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE_hive a join table_dictionary_include_hive b on a.imei=b.imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_342")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_343
-  test("TABLE_DICTIONARY_INCLUDE_343", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE a join table_dictionary_include b on a.gamepointid=b.gamepointid""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE_hive a join table_dictionary_include_hive b on a.gamepointid=b.gamepointid""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_343")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_344
-  test("TABLE_DICTIONARY_INCLUDE_344", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE a join table_dictionary_include b on a.Latest_Day=b.Latest_Day""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE_hive a join table_dictionary_include_hive b on a.Latest_Day=b.Latest_Day""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_344")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_345
-  test("TABLE_DICTIONARY_INCLUDE_345", Include) {
-
-    checkAnswer(s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE a join table_dictionary_include b on a.contractnumber=b.contractnumber""",
-      s"""select  b.contractNumber,b.Latest_DAY,b.gamePointId,b.productionDate,b.deviceInformationId,b.IMEI from TABLE_DICTIONARY_INCLUDE_hive a join table_dictionary_include_hive b on a.contractnumber=b.contractnumber""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_345")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_346
-  test("TABLE_DICTIONARY_INCLUDE_346", Include) {
-
-    checkAnswer(s"""select count( contractNumber ),sum( contractNumber ),count(distinct contractNumber ),avg( contractNumber ),max( contractNumber ),min( contractNumber ),1 from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count( contractNumber ),sum( contractNumber ),count(distinct contractNumber ),avg( contractNumber ),max( contractNumber ),min( contractNumber ),1 from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_346")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_347
-  test("TABLE_DICTIONARY_INCLUDE_347", Include) {
-
-    checkAnswer(s"""select count( Latest_Day ),sum( Latest_Day ),count(distinct Latest_Day ),avg( Latest_Day ),max( Latest_Day ),min( Latest_Day ),1 from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count( Latest_Day ),sum( Latest_Day ),count(distinct Latest_Day ),avg( Latest_Day ),max( Latest_Day ),min( Latest_Day ),1 from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_347")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_348
-  ignore("TABLE_DICTIONARY_INCLUDE_348", Include) {
-
-    checkAnswer(s"""select count( gamePointId),sum( gamePointId ),count(distinct gamePointId ),avg(gamePointId),max(gamePointId),min(gamePointId),1 from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count( gamePointId),sum( gamePointId ),count(distinct gamePointId ),avg(gamePointId),max(gamePointId),min(gamePointId),1 from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_348")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_349
-  test("TABLE_DICTIONARY_INCLUDE_349", Include) {
-
-    checkAnswer(s"""select count(deviceInformationId),sum(deviceInformationId),count(deviceInformationId),avg(deviceInformationId),max(deviceInformationId),min(deviceInformationId),1 from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(deviceInformationId),sum(deviceInformationId),count(deviceInformationId),avg(deviceInformationId),max(deviceInformationId),min(deviceInformationId),1 from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_349")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_350
-  test("TABLE_DICTIONARY_INCLUDE_350", Include) {
-
-    checkAnswer(s"""select count( productionDate),sum(  productionDate ),count(distinct  productionDate ),avg(  productionDate ),max(  productionDate),min(  productionDate ),1 from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count( productionDate),sum(  productionDate ),count(distinct  productionDate ),avg(  productionDate ),max(  productionDate),min(  productionDate ),1 from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_350")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_351
-  test("TABLE_DICTIONARY_INCLUDE_351", Include) {
-
-    checkAnswer(s"""select count(IMEI ),sum(IMEI ),count(distinct IMEI ),avg(IMEI ),max(IMEI ),min(IMEI ),1 from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(IMEI ),sum(IMEI ),count(distinct IMEI ),avg(IMEI ),max(IMEI ),min(IMEI ),1 from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_351")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_352
-  test("TABLE_DICTIONARY_INCLUDE_352", Include) {
-
-    checkAnswer(s"""select sum(contractNumber),count(contractNumber),avg(contractNumber),sum(contractNumber)/count(contractNumber) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(contractNumber),count(contractNumber),avg(contractNumber),sum(contractNumber)/count(contractNumber) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_352")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_353
-  test("TABLE_DICTIONARY_INCLUDE_353", Include) {
-
-    checkAnswer(s"""select sum(Latest_Day),count(Latest_Day),avg(Latest_Day),sum(Latest_Day)/count(Latest_Day) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(Latest_Day),count(Latest_Day),avg(Latest_Day),sum(Latest_Day)/count(Latest_Day) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_353")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_354
-  test("TABLE_DICTIONARY_INCLUDE_354", Include) {
-
-    checkAnswer(s"""select sum(gamepointId),count(gamepointId),avg(gamepointID),sum(gamepointID)/count(gamepointID) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(gamepointId),count(gamepointId),avg(gamepointID),sum(gamepointID)/count(gamepointID) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_354")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_355
-  test("TABLE_DICTIONARY_INCLUDE_355", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId),count(deviceinformationId),avg(deviceinformationId),sum(deviceinformationId)/count(deviceinformationId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(deviceinformationId),count(deviceinformationId),avg(deviceinformationId),sum(deviceinformationId)/count(deviceinformationId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_355")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_356
-  test("TABLE_DICTIONARY_INCLUDE_356", Include) {
-
-    checkAnswer(s"""select sum(IMEI),count(IMEI),avg(IMEI),sum(IMEI)/count(IMEI) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(IMEI),count(IMEI),avg(IMEI),sum(IMEI)/count(IMEI) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_356")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_357
-  test("TABLE_DICTIONARY_INCLUDE_357", Include) {
-
-    checkAnswer(s"""select sum(productionDate),count(productionDate),avg(productionDate),sum(productionDate)/count(productionDate) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select sum(productionDate),count(productionDate),avg(productionDate),sum(productionDate)/count(productionDate) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_357")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_358
-  test("TABLE_DICTIONARY_INCLUDE_358", Include) {
-
-    checkAnswer(s"""select contractNumber,Latest_DAY,gamePointId,productionDate,deviceInformationId,IMEI  from TABLE_DICTIONARY_INCLUDE""",
-      s"""select contractNumber,Latest_DAY,gamePointId,productionDate,deviceInformationId,IMEI  from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_358")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_359
-  test("TABLE_DICTIONARY_INCLUDE_359", Include) {
-
-    checkAnswer(s"""select count(MAC) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(MAC) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_359")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_360
-  test("TABLE_DICTIONARY_INCLUDE_360", Include) {
-
-    checkAnswer(s"""select count(gamePointId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(gamePointId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_360")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_361
-  test("TABLE_DICTIONARY_INCLUDE_361", Include) {
-
-    checkAnswer(s"""select count(contractNumber) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(contractNumber) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_361")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_362
-  test("TABLE_DICTIONARY_INCLUDE_362", Include) {
-
-    checkAnswer(s"""select count(Latest_DAY) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(Latest_DAY) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_362")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_363
-  test("TABLE_DICTIONARY_INCLUDE_363", Include) {
-
-    checkAnswer(s"""select count(productionDate) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(productionDate) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_363")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_364
-  test("TABLE_DICTIONARY_INCLUDE_364", Include) {
-
-    checkAnswer(s"""select count(deviceInformationId) from TABLE_DICTIONARY_INCLUDE""",
-      s"""select count(deviceInformationId) from TABLE_DICTIONARY_INCLUDE_hive""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_364")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_365
-  test("TABLE_DICTIONARY_INCLUDE_365", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  contractNumber  != '9223372047700'""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  contractNumber  != '9223372047700'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_365")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_366
-  test("TABLE_DICTIONARY_INCLUDE_366", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  Latest_DAY  != '1234567890123480.0000000000' order by imei,deviceInformationId,MAC,deviceColor limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  Latest_DAY  != '1234567890123480.0000000000' order by imei,deviceInformationId,MAC,deviceColor limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_366")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_367
-  test("TABLE_DICTIONARY_INCLUDE_367", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  gamePointId  != '2.27852521808948E36' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  gamePointId  != '2.27852521808948E36' order by imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_367")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_368
-  test("TABLE_DICTIONARY_INCLUDE_368", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  productionDate  != '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  productionDate  != '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_368")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_369
-  test("TABLE_DICTIONARY_INCLUDE_369", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  deviceInformationId  != '100075' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  deviceInformationId  != '100075' order by imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_369")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_370
-  test("TABLE_DICTIONARY_INCLUDE_370", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  contractNumber  not like '9223372047700' order by  imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  contractNumber  not like '9223372047700' order by  imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_370")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_371
-  test("TABLE_DICTIONARY_INCLUDE_371", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  Latest_DAY  not like '1234567890123480.0000000000' order by imei,deviceInformationId limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  Latest_DAY  not like '1234567890123480.0000000000' order by imei,deviceInformationId limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_371")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_372
-  test("TABLE_DICTIONARY_INCLUDE_372", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  gamePointId  not like '2.27852521808948E36' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  gamePointId  not like '2.27852521808948E36' order by imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_372")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_373
-  test("TABLE_DICTIONARY_INCLUDE_373", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  productionDate  not like '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  productionDate  not like '2015-09-18 12:07:28.0' order by imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_373")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_374
-  test("TABLE_DICTIONARY_INCLUDE_374", Include) {
-
-    checkAnswer(s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE where  deviceInformationId  not like '100075' order by imei,deviceInformationId,MAC limit 5""",
-      s"""select imei,deviceInformationId,MAC,deviceColor from TABLE_DICTIONARY_INCLUDE_hive where  deviceInformationId  not like '100075' order by imei,deviceInformationId,MAC limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_374")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_375
-  test("TABLE_DICTIONARY_INCLUDE_375", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei is not null""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei is not null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_375")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_376
-  test("TABLE_DICTIONARY_INCLUDE_376", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId is not null""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId is not null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_376")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_377
-  test("TABLE_DICTIONARY_INCLUDE_377", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber is not null""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber is not null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_377")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_378
-  test("TABLE_DICTIONARY_INCLUDE_378", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY is not null""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY is not null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_378")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_379
-  test("TABLE_DICTIONARY_INCLUDE_379", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate is not null""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate is not null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_379")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_380
-  test("TABLE_DICTIONARY_INCLUDE_380", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId is not null""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId is not null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_380")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_381
-  test("TABLE_DICTIONARY_INCLUDE_381", Include) {
-
-    checkAnswer(s"""select imei from TABLE_DICTIONARY_INCLUDE where imei is  null""",
-      s"""select imei from TABLE_DICTIONARY_INCLUDE_hive where imei is  null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_381")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_382
-  test("TABLE_DICTIONARY_INCLUDE_382", Include) {
-
-    checkAnswer(s"""select gamePointId from TABLE_DICTIONARY_INCLUDE where gamePointId is  null""",
-      s"""select gamePointId from TABLE_DICTIONARY_INCLUDE_hive where gamePointId is  null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_382")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_383
-  test("TABLE_DICTIONARY_INCLUDE_383", Include) {
-
-    checkAnswer(s"""select contractNumber from TABLE_DICTIONARY_INCLUDE where contractNumber is  null""",
-      s"""select contractNumber from TABLE_DICTIONARY_INCLUDE_hive where contractNumber is  null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_383")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_384
-  test("TABLE_DICTIONARY_INCLUDE_384", Include) {
-
-    checkAnswer(s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE where Latest_DAY is  null""",
-      s"""select Latest_DAY from TABLE_DICTIONARY_INCLUDE_hive where Latest_DAY is  null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_384")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_385
-  test("TABLE_DICTIONARY_INCLUDE_385", Include) {
-
-    checkAnswer(s"""select productionDate from TABLE_DICTIONARY_INCLUDE where productionDate is  null""",
-      s"""select productionDate from TABLE_DICTIONARY_INCLUDE_hive where productionDate is  null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_385")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_386
-  test("TABLE_DICTIONARY_INCLUDE_386", Include) {
-
-    checkAnswer(s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE where deviceInformationId is  null""",
-      s"""select deviceInformationId from TABLE_DICTIONARY_INCLUDE_hive where deviceInformationId is  null""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_386")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_387
-  test("TABLE_DICTIONARY_INCLUDE_387", Include) {
-
-    checkAnswer(s"""select count(*) from TABLE_DICTIONARY_INCLUDE where imei = '1AA1'""",
-      s"""select count(*) from TABLE_DICTIONARY_INCLUDE_hive where imei = '1AA1'""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_387")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_001
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_001", Include) {
-
-    checkAnswer(s"""select count(imei)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000'  and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(imei)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000'  and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_001")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_002
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_002", Include) {
-
-    checkAnswer(s"""select count(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_002")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_003
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_003", Include) {
-
-    checkAnswer(s"""select count(productionDate)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(productionDate)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_003")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_004
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_004", Include) {
-
-    checkAnswer(s"""select count(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(gamePointId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_004")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_005
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_005", Include) {
-
-    checkAnswer(s"""select count(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_005")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_006
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_006", Include) {
-
-    checkAnswer(s"""select count(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_006")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_007
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_007", Include) {
-
-    checkAnswer(s"""select count(distinct imei)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct imei)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site'  and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_007")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_008
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_008", Include) {
-
-    checkAnswer(s"""select count(distinct deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_008")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_009
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_009", Include) {
-
-    checkAnswer(s"""select count(distinct productionDate)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct productionDate)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_009")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_010
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_010", Include) {
-
-    checkAnswer(s"""select count(distinct gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct gamePointId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_010")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_011
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_011", Include) {
-
-    checkAnswer(s"""select count(distinct Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_011")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_012
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_012", Include) {
-
-    checkAnswer(s"""select count(distinct contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select count(distinct contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_012")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_013
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_013", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_013")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_014
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_014", Include) {
-
-    checkAnswer(s"""select sum(productionDate)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(productionDate)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_014")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_015
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_015", Include) {
-
-    sql(s"""select sum(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_016
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_016", Include) {
-
-    checkAnswer(s"""select sum(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_016")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_017
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_017", Include) {
-
-    checkAnswer(s"""select sum(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_017")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_018
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_018", Include) {
-
-    checkAnswer(s"""select sum(distinct deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_018")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_019
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_019", Include) {
-
-    sql(s"""select sum(distinct gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_020
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_020", Include) {
-
-    checkAnswer(s"""select sum(distinct Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_020")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_021
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_021", Include) {
-
-    checkAnswer(s"""select sum(distinct contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select sum(distinct contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_021")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_022
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_022", Include) {
-
-    checkAnswer(s"""select min(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_022")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_023
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_023", Include) {
-
-    checkAnswer(s"""select min(productionDate)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(productionDate)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_023")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_024
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_024", Include) {
-
-    checkAnswer(s"""select min(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(gamePointId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_024")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_025
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_025", Include) {
-
-    checkAnswer(s"""select min(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_025")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_026
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_026", Include) {
-
-    checkAnswer(s"""select min(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select min(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_026")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_027
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_027", Include) {
-
-    checkAnswer(s"""select max(imei)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(imei)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_027")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_028
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_028", Include) {
-
-    checkAnswer(s"""select max(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_028")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_029
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_029", Include) {
-
-    checkAnswer(s"""select max(productionDate)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(productionDate)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_029")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_030
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_030", Include) {
-
-    checkAnswer(s"""select max(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(gamePointId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_030")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_031
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_031", Include) {
-
-    checkAnswer(s"""select max(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_031")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_032
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_032", Include) {
-
-    checkAnswer(s"""select max(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select max(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_032")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_033
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_033", Include) {
-
-    checkAnswer(s"""select variance(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select variance(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_033")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_034
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_034", Include) {
-
-    checkAnswer(s"""select variance(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select variance(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_034")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_035
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_035", Include) {
-
-    checkAnswer(s"""select variance(gamePointId) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select variance(gamePointId) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_035")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_036
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_036", Include) {
-
-    checkAnswer(s"""select variance(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select variance(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_036")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_037
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_037", Include) {
-
-    checkAnswer(s"""select variance(contractNumber) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by contractNumber)""",
-      s"""select variance(contractNumber) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by contractNumber)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_037")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_038
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_038", Include) {
-
-    checkAnswer(s"""select var_samp(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select var_samp(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_038")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_039
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_039", Include) {
-
-    checkAnswer(s"""select var_samp(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select var_samp(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_039")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_040
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_040", Include) {
-
-    sql(s"""select var_samp(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_041
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_041", Include) {
-
-    checkAnswer(s"""select var_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select var_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_041")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_042
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_042", Include) {
-
-    checkAnswer(s"""select var_samp(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select var_samp(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_042")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_043
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_043", Include) {
-
-    checkAnswer(s"""select stddev_pop(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select stddev_pop(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_043")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_044
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_044", Include) {
-
-    checkAnswer(s"""select stddev_pop(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select stddev_pop(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_044")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_045
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_045", Include) {
-
-    sql(s"""select stddev_pop(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_046
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_046", Include) {
-
-    checkAnswer(s"""select stddev_pop(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select stddev_pop(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_046")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_047
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_047", Include) {
-
-    checkAnswer(s"""select stddev_pop(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select stddev_pop(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_047")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_048
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_048", Include) {
-
-    checkAnswer(s"""select stddev_samp(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select stddev_samp(deviceInformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_048")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_049
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_049", Include) {
-
-    checkAnswer(s"""select stddev_samp(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamepointid)""",
-      s"""select stddev_samp(gamepointid) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamepointid)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_049")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_050
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_050", Include) {
-
-    sql(s"""select stddev_samp(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_051
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_051", Include) {
-
-    checkAnswer(s"""select stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""",
-      s"""select stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by Latest_DAY)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_051")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_053
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_053", Include) {
-
-    checkAnswer(s"""select count(imei),count(distinct deviceinformationId),sum(deviceinformationId),sum(distinct deviceinformationId),min(deviceinformationId),max(imei),variance(gamePointId),var_samp(gamePointId),stddev_pop(gamePointId),stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""",
-      s"""select count(imei),count(distinct deviceinformationId),sum(deviceinformationId),sum(distinct deviceinformationId),min(deviceinformationId),max(imei),variance(gamePointId),var_samp(gamePointId),stddev_pop(gamePointId),stddev_samp(Latest_DAY) from (select * from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') order by gamePointId)""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_053")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_054
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_054", Include) {
-
-    checkAnswer(s"""select AVG(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(deviceinformationId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_054")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_055
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_055", Include) {
-
-    checkAnswer(s"""select AVG(productionDate)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(productionDate)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_055")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_056
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_056", Include) {
-
-    checkAnswer(s"""select AVG(gamePointId)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(gamePointId)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_056")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_057
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_057", Include) {
-
-    checkAnswer(s"""select AVG(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(Latest_DAY)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_057")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_058
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_058", Include) {
-
-    checkAnswer(s"""select AVG(contractNumber)  from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""",
-      s"""select AVG(contractNumber)  from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_058")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_059
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_059", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_INCLUDE group by deviceInformationId limit 5""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_060
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_060", Include) {
-
-    sql(s"""select sum(deviceinformationId),sum(distinct deviceinformationId),min(deviceinformationId),max(imei),variance(deviceInformationId) from TABLE_DICTIONARY_INCLUDE group by deviceInformationId order by deviceinformationId limit 5""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_061
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_061", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null')group by deviceInformationId limit 5""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_062
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_062", Include) {
-
-    checkAnswer(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') group by deviceInformationId order by deviceinformationId limit 5""",
-      s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_INCLUDE_hive where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') group by deviceInformationId order by deviceinformationId limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_062")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_063
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_063", Include) {
-
-    sql(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate),max(imei),variance(Latest_DAY) from TABLE_DICTIONARY_INCLUDE where deviceColor ='5Device Color' and modelId != '109' or Latest_DAY > '1234567890123540.0000000000' and contractNumber == '92233720368547800' or  Active_operaSysVersion like 'Operating System Version' and gamePointId <=> '8.1366141918611E39' and deviceInformationId < '1000000' and productionDate not like '2016-07-01' and imei is null and Latest_HOUR is not null and channelsId <= '7' and Latest_releaseId >= '1' and Latest_MONTH between 6 and 8 and Latest_YEAR not between 2016 and 2017 and Latest_HOUR RLIKE '12' and gamePointDescription REGEXP 'Site' and imei in ('1AA1','1AA100','1AA10','1AA1000','1AA10000','1AA100000','1AA1000000','1AA100001','1AA100002','1AA100004','','NULL') and Active_BacVerNumber not in ('Background version number1','','null') group by deviceInformationId,productionDate   sort by productionDate limit 5
-  """).collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_064
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_064", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId+10)t  from  TABLE_DICTIONARY_INCLUDE having t >1234567""",
-      s"""select sum(deviceinformationId+10)t  from  TABLE_DICTIONARY_INCLUDE_hive having t >1234567""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_064")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_065
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_065", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId+gamePointId)t  from  TABLE_DICTIONARY_INCLUDE having t >1234567""",
-      s"""select sum(deviceinformationId+gamePointId)t  from  TABLE_DICTIONARY_INCLUDE_hive having t >1234567""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_065")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_066
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_066", Include) {
-
-    checkAnswer(s"""select sum(deviceinformationId)t,Sum(gamePointId)   from  TABLE_DICTIONARY_INCLUDE having t >1234567""",
-      s"""select sum(deviceinformationId)t,Sum(gamePointId)   from  TABLE_DICTIONARY_INCLUDE_hive having t >1234567""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_066")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_067
-  ignore("TABLE_DICTIONARY_INCLUDE_PushUP_067", Include) {
-
-    checkAnswer(s"""select count(imei),sum(distinct deviceinformationId),min(productionDate)  from TABLE_DICTIONARY_INCLUDE group by imei,deviceinformationId,productionDate  order by  imei""",
-      s"""select count(imei),sum(distinct deviceinformationId),min(productionDate)  from TABLE_DICTIONARY_INCLUDE_hive group by imei,deviceinformationId,productionDate  order by  imei""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_067")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_069
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_069", Include) {
-
-    sql(s"""SELECT  min(Latest_DAY),max(imei),variance(contractNumber), SUM(gamePointId),count(imei),sum(distinct deviceinformationId) FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY GROUP BY AMSize, ActiveAreaId ORDER BY AMSize ASC, ActiveAreaId ASC limit 10""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_070
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_070", Include) {
-
-    checkAnswer(s"""SELECT  TABLE_DICTIONARY_INCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_INCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) TABLE_DICTIONARY_INCLUDE FULL OUTER JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) TABLE_DICTIONARY_INCLUDE1 ON TABLE_DICTIONARY_INCLUDE.AMSize = TABLE_DICTIONARY_INCLUDE1.AMSize WHERE TABLE_DICTIONARY_INCLUDE.AMSize LIKE '5RAM %' GROUP BY TABLE_DICTIONARY_INCLUDE.AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity,TABLE_DICTIONARY_INCLUDE.gamePointId ORDER BY TABLE_DICTIONARY_INCLUDE.gamePointId, TABLE_DICTIONARY_INCLUDE.AMSize ASC, TABLE_DICTIONARY_INCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_INCLUDE.Activecity ASC""",
-      s"""SELECT  TABLE_DICTIONARY_INCLUDE_hive.gamePointId AS gamePointId,TABLE_DICTIONARY_INCLUDE_hive.AMSize AS AMSize, TABLE_DICTIONARY_INCLUDE_hive.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_INCLUDE_hive.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_INCLUDE_hive) SUB_QRY ) TABLE_DICTIONARY_INCLUDE_hive FULL OUTER JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_INCLUDE_hive) SUB_QRY ) TABLE_DICTIONARY_INCLUDE_hive1 ON TABLE_DICTIONARY_INCLUDE_hive.AMSize = TABLE_DICTIONARY_INCLUDE_hive1.AMSize WHERE TABLE_DICTIONARY_INCLUDE_hive.AMSize LIKE '5RAM %' GROUP BY TABLE_DICTIONARY_INCLUDE_hive.AMSize, TABLE_DICTIONARY_INCLUDE_hive.ActiveCountry, TABLE_DICTIONARY_INCLUDE_hive.Activecity,TABLE_DICTIONARY_INCLUDE_hive.gamePointId ORDER BY TABLE_DICTIONARY_INCLUDE_hive.gamePointId, TABLE_DICTIONARY_INCLUDE_hive.AMSize ASC, TABLE_DICTIONARY_INCLUDE_hive.ActiveCountry ASC, TABLE_DICTIONARY_INCLUDE_hive.Activecity ASC""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_070")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_071
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_071", Include) {
-
-    sql(s"""SELECT TABLE_DICTIONARY_INCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_INCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) TABLE_DICTIONARY_INCLUDE RIGHT JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) Carbon_automation1 ON TABLE_DICTIONARY_INCLUDE.AMSize = Carbon_automation1.AMSize WHERE NOT(TABLE_DICTIONARY_INCLUDE.AMSize = "8RAM size") GROUP BY TABLE_DICTIONARY_INCLUDE.AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity,TABLE_DICTIONARY_INCLUDE.gamePointId  ORDER BY TABLE_DICTIONARY_INCLUDE.AMSize ASC, TABLE_DICTIONARY_INCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_INCLUDE.Activecity ASC limit 10""").collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_072
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_072", Include) {
-
-    sql(s"""SELECT TABLE_DICTIONARY_INCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_INCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) TABLE_DICTIONARY_INCLUDE LEFT JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) Carbon_automation1 ON TABLE_DICTIONARY_INCLUDE.AMSize = Carbon_automation1.AMSize WHERE NOT(TABLE_DICTIONARY_INCLUDE.AMSize = "8RAM size") GROUP BY TABLE_DICTIONARY_INCLUDE.AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity,TABLE_DICTIONARY_INCLUDE.gamePointId  ORDER BY TABLE_DICTIONARY_INCLUDE.AMSize ASC, TABLE_DICTIONARY_INCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_INCLUDE.Activecity ASC limit 10
-  """).collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_073
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_073", Include) {
-
-    sql(s"""SELECT TABLE_DICTIONARY_INCLUDE.gamePointId AS gamePointId,TABLE_DICTIONARY_INCLUDE.AMSize AS AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry AS ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity AS Activecity FROM ( SELECT AMSize,gamePointId, ActiveCountry, Activecity FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) TABLE_DICTIONARY_INCLUDE INNER JOIN ( SELECT ActiveCountry, Activecity, AMSize FROM (select * from TABLE_DICTIONARY_INCLUDE) SUB_QRY ) Carbon_automation1 ON TABLE_DICTIONARY_INCLUDE.AMSize = Carbon_automation1.AMSize WHERE NOT(TABLE_DICTIONARY_INCLUDE.AMSize = "8RAM size") GROUP BY TABLE_DICTIONARY_INCLUDE.AMSize, TABLE_DICTIONARY_INCLUDE.ActiveCountry, TABLE_DICTIONARY_INCLUDE.Activecity,TABLE_DICTIONARY_INCLUDE.gamePointId  ORDER BY TABLE_DICTIONARY_INCLUDE.AMSize ASC, TABLE_DICTIONARY_INCLUDE.ActiveCountry ASC, TABLE_DICTIONARY_INCLUDE.Activecity ASC limit 10
-  """).collect
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_074
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_074", Include) {
-
-    checkAnswer(s"""select count(gamepointid),series  from TABLE_DICTIONARY_INCLUDE group by series order by series limit 5""",
-      s"""select count(gamepointid),series  from TABLE_DICTIONARY_INCLUDE_hive group by series order by series limit 5""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_074")
-
-  }
-
-
-  //TABLE_DICTIONARY_INCLUDE_PushUP_075
-  test("TABLE_DICTIONARY_INCLUDE_PushUP_075", Include) {
-
-    checkAnswer(s"""select count(gamepointid),series  from TABLE_DICTIONARY_INCLUDE group by series order by series""",
-      s"""select count(gamepointid),series  from TABLE_DICTIONARY_INCLUDE_hive group by series order by series""", "QueriesIncludeDictionaryTestCase_DICTIONARY_INCLUDE_PushUP_075")
-
-  }
-
-
-  //CreateTable-DICTIONARY_INCLUDE-01
-  test("CreateTable-DICTIONARY_INCLUDE-01", Include) {
-    sql(s"""drop table if exists smart_500_DINC""").collect
-    sql(s"""drop table if exists smart_500_DINC_hive""").collect
-
-    sql(s"""create table smart_500_DINC (MSISDN string,IMSI string,IMEI string,INTERFACEID int,GROUPID int,GGSN_ID double,SGSN_ID double,SESSION_INDICATOR int,BEGIN_TIME double,BEGIN_TIME_MSEL int,END_TIME double,END_TIME_MSEL int,PROT_CATEGORY int,PROT_TYPE int,L7_CARRIER_PROT int,SUB_PROT_TYPE int,SID double,PROBEID double,ENCRYPT_VERSION int,ROAMING_TYPE int,ROAM_DIRECTION int,MS_IP string,SERVER_IP string,MS_PORT int,SERVER_DECIMAL Decimal,APN string,SGSN_SIG_IP string,GGSN_SIG_IP_BigInt_NEGATIVE bigint,SGSN_USER_IP string,GGSN_USER_IP string,MCC string,MNC string,RAT int,LAC string,RAC string,SAC string,CI string,BROWSER_TIMESTAMP timestamp,TCP_CONN_STATES int,TCP_STATES_BIGINTPOSITIVE int,TCP_WIN_SIZE int,TCP_MSS int,TCP_CONN_TIMES int,TCP_CONN_2_FAILED_TIMES int,TCP_CONN_3_FAILED_TIMES int,HOST string,STREAMING_URL string,GET_STREAMING_FAILED_CODE int,GET_STREAMING_FLAG int,GET_NUM int,GET_SUCCEED_NUM int,GET_RETRANS_NUM int,GET_TIMEOUT_NUM int,INTBUFFER_FST_FLAG int,INTBUFFER_FULL_FLAG int,STALL_NUM int,VIDEO_FRAME_RATE int,VIDEO_CODEC_ID string,VIDEO_WIDTH int,VIDEO_HEIGHT int,AUDIO_CODEC_ID string,MEDIA_FILE_TYPE int,PLAY_STATE int,STREAMING_FLAG int,TCP_STATUS_INDICATOR int,DISCONNECTION_FLAG int,FAILURE_CODE int,FLAG int,TAC string,ECI string,TCP_SYN_TIME_MSEL int,TCP_FST_SYN_DIRECTION int,RAN_NE_USER_IP string,HOMEMCC string,HOMEMNC string,CHARGE_FLAG int,PREPAID_FLAG int,USER_AGENT string,MS_WIN_STAT_TOTAL_NUM int,MS_WIN_STAT_SMALL_NUM int,MS_ACK_TO_1STGET_DELAY int,SERVER_ACK_TO_1STDATA_DELAY int,STREAMING_TYPE int,SOURCE_VIDEO_QUALITY int,TETHERING_FLAG int,CARRIER_ID double,LAYER1ID int,LAYER2ID int,LAYER3ID int,LAYER4ID int,LAYER5ID int,LAYER6ID int,CHARGING_RULE_BASE_NAME string,SP string,EXTENDED_URL string,SV string,FIRST_SAI_CGI_ECGI string,EXTENDED_URL_OTHER string,SIGNALING_USE_FLAG int,DNS_RETRANS_NUM int,DNS_FAIL_CODE int,FIRST_RAT int,MS_INDICATOR string,LAST_SAI_CGI_ECGI string,LAST_RAT int,FIRST_LONGITUDE double,FIRST_LATITUDE double,FIRST_ALTITUDE int,FIRST_RASTERLONGITUDE double,FIRST_RASTERLATITUDE double,FIRST_RASTERALTITUDE int,FIRST_FREQUENCYSPOT int,FIRST_CLUTTER int,FIRST_USERBEHAVIOR int,FIRST_SPEED int,FIRST_CREDIBILITY int,LAST_LONGITUDE double,LAST_LATITUDE double,LAST_ALTITUDE int,LAST_RASTERLONGITUDE double,LAST_RASTERLATITUDE double,LAST_RASTERALTITUDE int,LAST_FREQUENCYSPOT int,LAST_CLUTTER int,LAST_USERBEHAVIOR int,LAST_SPEED int,LAST_CREDIBILITY int,IMEI_CIPHERTEXT string,APP_ID int,DOMAIN_NAME string,STREAMING_CACHE_IP string,STOP_LONGER_THAN_MIN_THRESHOLD int,STOP_LONGER_THAN_MAX_THRESHOLD int,PLAY_END_STAT int,STOP_START_TIME1 double,STOP_END_TIME1 double,STOP_START_TIME2 double,STOP_END_TIME2 double,STOP_START_TIME3 double,STOP_END_TIME3 double,STOP_START_TIME4 double,STOP_END_TIME4 double,STOP_START_TIME5 double,STOP_END_TIME5 double,STOP_START_TIME6 double,STOP_END_TIME6 double,STOP_START_TIME7 double,STOP_END_TIME7 double,STOP_START_TIME8 double,STOP_END_TIME8 double,STOP_START_TIME9 double,STOP_END_TIME9 double,STOP_START_TIME10 double,STOP_END_TIME10 double,FAIL_CLASS double,RECORD_TYPE double,NODATA_COUNT double,VIDEO_NODATA_DURATION double,VIDEO_SMOOTH_DURATION double,VIDEO_SD_DURATION double,VIDEO_HD_DURATION double,VIDEO_UHD_DURATION double,VIDEO_FHD_DURATION double,FLUCTUATION double,START_DOWNLOAD_THROUGHPUT double,L7_UL_GOODPUT_FULL_MSS double,SESSIONKEY string,FIRST_UCELLID double,LAST_UCELLID double,UCELLID1 double,LONGITUDE1 double,LATITUDE1 double,UCELLID2 double,LONGITUDE2 double,LATITUDE2 double,UCELLID3 double,LONGITUDE3 double,LATITUDE3 double,UCELLID4 double,LONGITUDE4 double,LATITUDE4 double,UCELLID5 double,LONGITUDE5 double,LATITUDE5 double,UCELLID6 double,LONGITUDE6 double,LATITUDE6 double,UCELLID7 double,LONGITUDE7 double,LATITUDE7 double,UCELLID8 double,LONGITUDE8 double,LATITUDE8 double,UCELLID9 double,LONGITUDE9 double,LATITUDE9 double,UCELLID10 double,LONGITUDE10 double,LATITUDE10 double,INTBUFFER_FULL_DELAY double,STALL_DURATION double,STREAMING_DW_PACKETS double,STREAMING_DOWNLOAD_DELAY double,PLAY_DURATION double,STREAMING_QUALITY int,VIDEO_DATA_RATE double,AUDIO_DATA_RATE double,STREAMING_FILESIZE double,STREAMING_DURATIOIN double,TCP_SYN_TIME double,TCP_RTT_STEP1 double,CHARGE_ID double,UL_REVERSE_TO_DL_DELAY double,DL_REVERSE_TO_UL_DELAY double,DATATRANS_DW_GOODPUT double,DATATRANS_DW_TOTAL_DURATION double,SUM_FRAGMENT_INTERVAL double,TCP_FIN_TIMES double,TCP_RESET_TIMES double,URL_CLASSIFICATION double,STREAMING_LQ_DURATIOIN double,MAX_DNS_DELAY double,MAX_DNS2SYN double,MAX_LATANCY_OF_LINK_SETUP double,MAX_SYNACK2FIRSTACK double,MAX_SYNACK2LASTACK double,MAX_ACK2GET_DELAY double,MAX_FRAG_INTERVAL_PREDELAY double,SUM_FRAG_INTERVAL_PREDELAY double,SERVICE_DELAY_MSEC double,HOMEPROVINCE double,HOMECITY double,SERVICE_ID double,CHARGING_CLASS double,DATATRANS_UL_DURATION double,ASSOCIATED_ID double,PACKET_LOSS_NUM double,JITTER double,MS_DNS_DELAY_MSEL double,GET_STREAMING_DELAY double,TCP_UL_RETRANS_WITHOUTPL double,TCP_DW_RETRANS_WITHOUTPL double,GET_MAX_UL_SIZE double,GET_MIN_UL_SIZE double,GET_MAX_DL_SIZE double,GET_MIN_DL_SIZE double,L4_UL_THROUGHPUT double,L4_DW_THROUGHPUT double,L4_UL_GOODPUT double,L4_DW_GOODPUT double,NETWORK_UL_TRAFFIC double,NETWORK_DL_TRAFFIC double,L4_UL_PACKETS double,L4_DW_PACKETS double,TCP_RTT double,TCP_UL_OUTOFSEQU double,TCP_DW_OUTOFSEQU double,TCP_UL_RETRANS double,TCP_DW_RETRANS double,TCP_UL_RETRANS_WITHPL double,TCP_DW_RETRANS_WITHPL double,TCP_UL_PACKAGES_WITHPL double,TCP_DW_PACKAGES_WITHPL double,TCP_UL_PACKAGES_WITHOUTPL double,TCP_DW_PACKAGES_WITHOUTPL double,UPPERLAYER_IP_UL_PACKETS double,UPPERLAYER_IP_DL_PACKETS double,DOWNLAYER_IP_UL_PACKETS double,DOWNLAYER_IP_DL_PACKETS double,UPPERLAYER_IP_UL_FRAGMENTS double,UPPERLAYER_IP_DL_FRAGMENTS double,DOWNLAYER_IP_UL_FRAGMENTS double,DOWNLAYER_IP_DL_FRAGMENTS double,VALID_TRANS_DURATION double,AIR_PORT_DURATION double,RADIO_CONN_TIMES double,RAN_NE_ID double,AVG_UL_RTT double,AVG_DW_RTT double,UL_RTT_LONG_NUM int,DW_RTT_LONG_NUM int,UL_RTT_STAT_NUM int,DW_RTT_STAT_NUM int,USER_PROBE_UL_LOST_PKT int,SERVER_PROBE_UL_LOST_PKT int,SERVER_PROBE_DW_LOST_PKT int,USER_PROBE_DW_LOST_PKT int,CHARGING_CHARACTERISTICS double,DL_SERIOUS_OUT_OF_ORDER_NUM double,DL_SLIGHT_OUT_OF_ORDER_NUM double,DL_FLIGHT_TOTAL_SIZE double,DL_FLIGHT_TOTAL_NUM double,DL_MAX_FLIGHT_SIZE double,UL_SERIOUS_OUT_OF_ORDER_NUM double,UL_SLIGHT_OUT_OF_ORDER_NUM double,UL_FLIGHT_TOTAL_SIZE double,UL_FLIGHT_TOTAL_NUM double,UL_MAX_FLIGHT_SIZE double,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS double,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS double,DL_CONTINUOUS_RETRANSMISSION_DELAY double,USER_HUNGRY_DELAY double,SERVER_HUNGRY_DELAY double,AVG_DW_RTT_MICRO_SEC int,AVG_UL_RTT_MICRO_SEC int,FLOW_SAMPLE_RATIO int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ( 'DICTIONARY_INCLUDE'='BEGIN_TIME,END_TIME,SID,MSISDN,IMSI,IMEI,MS_IP,SERVER_IP,HOST,SP,MS_INDICATOR,streaming_url,LAYER1ID,TCP_DW_RETRANS')""").collect
-
-    sql(s"""create table smart_500_DINC_hive (SID double,PROBEID double,INTERFACEID int,GROUPID int,GGSN_ID double,SGSN_ID double,dummy_6 string,SESSION_INDICATOR int,BEGIN_TIME double,BEGIN_TIME_MSEL int,END_TIME double,END_TIME_MSEL int,PROT_CATEGORY int,PROT_TYPE int,L7_CARRIER_PROT int,SUB_PROT_TYPE int,MSISDN string,IMSI string,IMEI string,ENCRYPT_VERSION int,ROAMING_TYPE int,ROAM_DIRECTION int,MS_IP string,SERVER_IP string,MS_PORT int,APN string,SGSN_SIG_IP string,GGSN_USER_IP string,SGSN_USER_IP string,MCC string,MNC string,RAT int,LAC string,RAC string,SAC string,CI string,SERVER_DECIMAL decimal,BROWSER_TIMESTAMP timestamp,TCP_CONN_STATES int,GGSN_SIG_IP_BigInt_NEGATIVE bigint,TCP_STATES_BIGINTPOSITIVE bigint,dummy_41 string,TCP_WIN_SIZE int,dummy_43 string,TCP_MSS int,dummy_45 string,TCP_CONN_TIMES int,dummy_47 string,TCP_CONN_2_FAILED_TIMES int,dummy_49 string,TCP_CONN_3_FAILED_TIMES int,HOST string,STREAMING_URL string,dummy_53 string,GET_STREAMING_FAILED_CODE int,dummy_55 string,GET_STREAMING_FLAG int,dummy_57 string,GET_NUM int,dummy_59 string,GET_SUCCEED_NUM int,dummy_61 string,GET_RETRANS_NUM int,dummy_63 string,GET_TIMEOUT_NUM int,INTBUFFER_FST_FLAG int,INTBUFFER_FULL_FLAG int,STALL_NUM int,dummy_68 string,VIDEO_FRAME_RATE int,dummy_70 string,VIDEO_CODEC_ID string,dummy_72 string,VIDEO_WIDTH int,dummy_74 string,VIDEO_HEIGHT int,dummy_76 string,AUDIO_CODEC_ID string,dummy_78 string,MEDIA_FILE_TYPE int,dummy_80 string,PLAY_STATE int,dummy_82 string,PLAY_STATE_1 int,dummy_84 string,STREAMING_FLAG int,dummy_86 string,TCP_STATUS_INDICATOR int,dummy_88 string,DISCONNECTION_FLAG int,dummy_90 string,FAILURE_CODE int,FLAG int,TAC string,ECI string,dummy_95 string,TCP_SYN_TIME_MSEL int,dummy_97 string,TCP_FST_SYN_DIRECTION int,RAN_NE_USER_IP string,HOMEMCC string,HOMEMNC string,dummy_102 string,CHARGE_FLAG int,dummy_104 string,PREPAID_FLAG int,dummy_106 string,USER_AGENT string,dummy_108 string,MS_WIN_STAT_TOTAL_NUM int,dummy_110 string,MS_WIN_STAT_SMALL_NUM int,dummy_112 string,MS_ACK_TO_1STGET_DELAY int,dummy_114 string,SERVER_ACK_TO_1STDATA_DELAY int,dummy_116 string,STREAMING_TYPE int,dummy_118 string,SOURCE_VIDEO_QUALITY int,TETHERING_FLAG int,CARRIER_ID double,LAYER1ID int,LAYER2ID int,dummy_124 string,LAYER3ID int,dummy_126 string,LAYER4ID int,dummy_128 string,LAYER5ID int,dummy_130 string,LAYER6ID int,CHARGING_RULE_BASE_NAME string,SP string,dummy_134 string,EXTENDED_URL string,SV string,FIRST_SAI_CGI_ECGI string,dummy_138 string,EXTENDED_URL_OTHER string,SIGNALING_USE_FLAG int,dummy_141 string,DNS_RETRANS_NUM int,dummy_143 string,DNS_FAIL_CODE int,FIRST_RAT int,FIRST_RAT_1 int,MS_INDICATOR string,LAST_SAI_CGI_ECGI string,LAST_RAT int,dummy_150 string,FIRST_LONGITUDE double,dummy_152 string,FIRST_LATITUDE double,dummy_154 string,FIRST_ALTITUDE int,dummy_156 string,FIRST_RASTERLONGITUDE double,dummy_158 string,FIRST_RASTERLATITUDE double,dummy_160 string,FIRST_RASTERALTITUDE int,dummy_162 string,FIRST_FREQUENCYSPOT int,dummy_164 string,FIRST_CLUTTER int,dummy_166 string,FIRST_USERBEHAVIOR int,dummy_168 string,FIRST_SPEED int,dummy_170 string,FIRST_CREDIBILITY int,dummy_172 string,LAST_LONGITUDE double,dummy_174 string,LAST_LATITUDE double,dummy_176 string,LAST_ALTITUDE int,dummy_178 string,LAST_RASTERLONGITUDE double,dummy_180 string,LAST_RASTERLATITUDE double,dummy_182 string,LAST_RASTERALTITUDE int,dummy_184 string,LAST_FREQUENCYSPOT int,dummy_186 string,LAST_CLUTTER int,dummy_188 string,LAST_USERBEHAVIOR int,dummy_190 string,LAST_SPEED int,dummy_192 string,LAST_CREDIBILITY int,dummy_194 string,IMEI_CIPHERTEXT string,APP_ID int,dummy_197 string,DOMAIN_NAME string,dummy_199 string,STREAMING_CACHE_IP string,dummy_201 string,STOP_LONGER_THAN_MIN_THRESHOLD int,dummy_203 string,STOP_LONGER_THAN_MAX_THRESHOLD int,dummy_205 string,PLAY_END_STAT int,dummy_207 string,STOP_START_TIME1 double,dummy_209 string,STOP_END_TIME1 double,dummy_211 string,STOP_START_TIME2 double,dummy_213 string,STOP_END_TIME2 double,dummy_215 string,STOP_START_TIME3 double,dummy_217 string,STOP_END_TIME3 double,dummy_219 string,STOP_START_TIME4 double,dummy_221 string,STOP_END_TIME4 double,dummy_223 string,STOP_START_TIME5 double,dummy_225 string,STOP_END_TIME5 double,dummy_227 string,STOP_START_TIME6 double,dummy_229 string,STOP_END_TIME6 double,dummy_231 string,STOP_START_TIME7 double,dummy_233 string,STOP_END_TIME7 double,dummy_235 string,STOP_START_TIME8 double,dummy_237 string,STOP_END_TIME8 double,dummy_239 string,STOP_START_TIME9 double,dummy_241 string,STOP_END_TIME9 double,dummy_243 string,STOP_START_TIME10 double,dummy_245 string,STOP_END_TIME10 double,dummy_247 string,FAIL_CLASS double,RECORD_TYPE double,dummy_250 string,NODATA_COUNT double,dummy_252 string,VIDEO_NODATA_DURATION double,dummy_254 string,VIDEO_SMOOTH_DURATION double,dummy_256 string,VIDEO_SD_DURATION double,dummy_258 string,VIDEO_HD_DURATION double,dummy_260 string,VIDEO_UHD_DURATION double,dummy_262 string,VIDEO_FHD_DURATION double,dummy_264 string,FLUCTUATION double,dummy_266 string,START_DOWNLOAD_THROUGHPUT double,dummy_268 string,L7_UL_GOODPUT_FULL_MSS double,dummy_270 string,SESSIONKEY string,dummy_272 string,FIRST_UCELLID double,dummy_274 string,LAST_UCELLID double,dummy_276 string,UCELLID1 double,dummy_278 string,LONGITUDE1 double,dummy_280 string,LATITUDE1 double,dummy_282 string,UCELLID2 double,dummy_284 string,LONGITUDE2 double,dummy_286 string,LATITUDE2 double,dummy_288 string,UCELLID3 double,dummy_290 string,LONGITUDE3 double,dummy_292 string,LATITUDE3 double,dummy_294 string,UCELLID4 double,dummy_296 string,LONGITUDE4 double,dummy_2101 string,LATITUDE4 double,dummy_300 string,UCELLID5 double,dummy_302 string,LONGITUDE5 double,dummy_304 string,LATITUDE5 double,dummy_306 string,UCELLID6 double,dummy_308 string,LONGITUDE6 double,dummy_310 string,LATITUDE6 double,dummy_312 string,UCELLID7 double,dummy_314 string,LONGITUDE7 double,dummy_316 string,LATITUDE7 double,dummy_318 string,UCELLID8 double,dummy_320 string,LONGITUDE8 double,dummy_322 string,LATITUDE8 double,dummy_324 string,UCELLID9 double,dummy_326 string,LONGITUDE9 double,dummy_328 string,LATITUDE9 double,dummy_330 string,UCELLID10 double,dummy_332 string,LONGITUDE10 double,dummy_334 string,LATITUDE10 double,dummy_336 string,INTBUFFER_FULL_DELAY double,dummy_338 string,STALL_DURATION double,dummy_340 string,STREAMING_DW_PACKETS double,dummy_342 string,STREAMING_DOWNLOAD_DELAY double,dummy_344 string,PLAY_DURATION double,dummy_346 string,STREAMING_QUALITY int,dummy_348 string,VIDEO_DATA_RATE double,dummy_350 string,AUDIO_DATA_RATE double,dummy_352 string,STREAMING_FILESIZE double,dummy_354 string,STREAMING_DURATIOIN double,dummy_356 string,TCP_SYN_TIME double,dummy_358 string,TCP_RTT_STEP1 double,CHARGE_ID double,dummy_361 string,UL_REVERSE_TO_DL_DELAY double,dummy_363 string,DL_REVERSE_TO_UL_DELAY double,dummy_365 string,DATATRANS_DW_GOODPUT double,dummy_367 string,DATATRANS_DW_TOTAL_DURATION double,dummy_369 string,SUM_FRAGMENT_INTERVAL double,dummy_371 string,TCP_FIN_TIMES double,dummy_373 string,TCP_RESET_TIMES double,dummy_375 string,URL_CLASSIFICATION double,dummy_377 string,STREAMING_LQ_DURATIOIN double,dummy_379 string,MAX_DNS_DELAY double,dummy_381 string,MAX_DNS2SYN double,dummy_383 string,MAX_LATANCY_OF_LINK_SETUP double,dummy_385 string,MAX_SYNACK2FIRSTACK double,dummy_387 string,MAX_SYNACK2LASTACK double,dummy_389 string,MAX_ACK2GET_DELAY double,dummy_391 string,MAX_FRAG_INTERVAL_PREDELAY double,dummy_393 string,SUM_FRAG_INTERVAL_PREDELAY double,dummy_395 string,SERVICE_DELAY_MSEC double,dummy_397 string,HOMEPROVINCE double,dummy_399 string,HOMECITY double,dummy_401 string,SERVICE_ID double,dummy_403 string,CHARGING_CLASS double,dummy_405 string,DATATRANS_UL_DURATION double,dummy_407 string,ASSOCIATED_ID double,dummy_409 string,PACKET_LOSS_NUM double,dummy_411 string,JITTER double,dummy_413 string,MS_DNS_DELAY_MSEL double,dummy_415 string,GET_STREAMING_DELAY double,dummy_417 string,TCP_UL_RETRANS_WITHOUTPL double,dummy_419 string,TCP_DW_RETRANS_WITHOUTPL double,dummy_421 string,GET_MAX_UL_SIZE double,dummy_423 string,GET_MIN_UL_SIZE double,dummy_425 string,GET_MAX_DL_SIZE double,dummy_427 string,GET_MIN_DL_SIZE double,dummy_429 string,FLOW_SAMPLE_RATIO int,dummy_431 string,UL_RTT_LONG_NUM int,dummy_433 string,DW_RTT_LONG_NUM int,dummy_435 string,UL_RTT_STAT_NUM int,dummy_437 string,DW_RTT_STAT_NUM int,dummy_439 string,USER_PROBE_UL_LOST_PKT int,dummy_441 string,SERVER_PROBE_UL_LOST_PKT int,dummy_443 string,SERVER_PROBE_DW_LOST_PKT int,dummy_445 string,USER_PROBE_DW_LOST_PKT int,dummy_447 string,AVG_DW_RTT_MICRO_SEC int,dummy_449 string,AVG_UL_RTT_MICRO_SEC int,dummy_451 string,RAN_NE_ID double,dummy_453 string,AVG_UL_RTT double,dummy_455 string,AVG_DW_RTT double,dummy_457 string,CHARGING_CHARACTERISTICS double,dummy_459 string,DL_SERIOUS_OUT_OF_ORDER_NUM double,dummy_461 string,DL_SLIGHT_OUT_OF_ORDER_NUM double,dummy_463 string,DL_FLIGHT_TOTAL_SIZE double,dummy_465 string,DL_FLIGHT_TOTAL_NUM double,dummy_467 string,DL_MAX_FLIGHT_SIZE double,dummy_469 string,VALID_TRANS_DURATION double,dummy_471 string,AIR_PORT_DURATION double,dummy_473 string,RADIO_CONN_TIMES double,dummy_475 string,UL_SERIOUS_OUT_OF_ORDER_NUM double,dummy_477 string,UL_SLIGHT_OUT_OF_ORDER_NUM double,dummy_479 string,UL_FLIGHT_TOTAL_SIZE double,dummy_481 string,UL_FLIGHT_TOTAL_NUM double,dummy_483 string,UL_MAX_FLIGHT_SIZE double,dummy_485 string,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS double,dummy_487 string,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS double,dummy_489 string,DL_CONTINUOUS_RETRANSMISSION_DELAY double,dummy_491 string,USER_HUNGRY_DELAY double,dummy_493 string,SERVER_HUNGRY_DELAY double,dummy_495 string,UPPERLAYER_IP_UL_FRAGMENTS double,dummy_497 string,UPPERLAYER_IP_DL_FRAGMENTS double,dummy_499 string,DOWNLAYER_IP_UL_FRAGMENTS double,dummy_501 string,DOWNLAYER_IP_DL_FRAGMENTS double,dummy_503 string,UPPERLAYER_IP_UL_PACKETS double,dummy_505 string,UPPERLAYER_IP_DL_PACKETS double,dummy_507 string,DOWNLAYER_IP_UL_PACKETS double,dummy_509 string,DOWNLAYER_IP_DL_PACKETS double,dummy_511 string,TCP_UL_PACKAGES_WITHPL double,dummy_513 string,TCP_DW_PACKAGES_WITHPL double,dummy_515 string,TCP_UL_PACKAGES_WITHOUTPL double,dummy_517 string,TCP_DW_PACKAGES_WITHOUTPL double,dummy_519 string,TCP_UL_RETRANS_WITHPL double,dummy_521 string,TCP_DW_RETRANS_WITHPL double,L4_UL_THROUGHPUT double,L4_DW_THROUGHPUT double,L4_UL_GOODPUT double,L4_DW_GOODPUT double,NETWORK_UL_TRAFFIC double,NETWORK_DL_TRAFFIC double,L4_UL_PACKETS double,L4_DW_PACKETS double,TCP_RTT double,TCP_UL_OUTOFSEQU double,TCP_DW_OUTOFSEQU double,TCP_UL_RETRANS double,TCP_DW_RETRANS double) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
-
-
-  }
-
-
-  //Dataload-DICTIONARY_INCLUDE-01
-  test("Dataload-DICTIONARY_INCLUDE-01", Include) {
-
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/SEQ500/seq_500Records.csv' into table smart_500_DINC options('DELIMITER'=',', 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='SID,PROBEID,INTERFACEID,GROUPID,GGSN_ID,SGSN_ID,dummy,SESSION_INDICATOR,BEGIN_TIME,BEGIN_TIME_MSEL,END_TIME,END_TIME_MSEL,PROT_CATEGORY,PROT_TYPE,L7_CARRIER_PROT,SUB_PROT_TYPE,MSISDN,IMSI,IMEI,ENCRYPT_VERSION,ROAMING_TYPE,ROAM_DIRECTION,MS_IP,SERVER_IP,MS_PORT,APN,SGSN_SIG_IP,GGSN_USER_IP,SGSN_USER_IP,MCC,MNC,RAT,LAC,RAC,SAC,CI,SERVER_DECIMAL,BROWSER_TIMESTAMP,TCP_CONN_STATES,GGSN_SIG_IP_BigInt_NEGATIVE,TCP_STATES_BIGINTPOSITIVE,dummy,TCP_WIN_SIZE,dummy,TCP_MSS,dummy,TCP_CONN_TIMES,dummy,TCP_CONN_2_FAILED_TIMES,dummy,TCP_CONN_3_FAILED_TIMES,HOST,STREAMING_URL,dummy,GET_STREAMING_FAILED_CODE,dummy,GET_STREAMING_FLAG,dummy,GET_NUM,dummy,GET_SUCCEED_NUM,dummy,GET_RETRANS_NUM,dummy,GET_TIMEOUT_NUM,INTBUFFER_FST_FLAG,INTBUFFER_FULL_FLAG,STALL_NUM,dummy,VIDEO_FRAME_RATE,dummy,VIDEO_CODEC_ID,dummy,VIDEO_WIDTH,dummy,VIDEO_HEIGHT,dummy,AUDIO_CODEC_ID,dummy,MEDIA_FILE_TYPE,dummy,PLAY_STATE,dummy,PLAY_STATE,dummy,STREAMING_FLAG,dummy,TCP_STATUS_INDICATOR,dummy,DISCONNECTION_FLAG,dummy,FAILURE_CODE,FLAG,TAC,ECI,dummy,TCP_SYN_TIME_MSEL,dummy,TCP_FST_SYN_DIRECTION,RAN_NE_USER_IP,HOMEMCC,HOMEMNC,dummy,CHARGE_FLAG,dummy,PREPAID_FLAG,dummy,USER_AGENT,dummy,MS_WIN_STAT_TOTAL_NUM,dummy,MS_WIN_STAT_SMALL_NUM,dummy,MS_ACK_TO_1STGET_DELAY,dummy,SERVER_ACK_TO_1STDATA_DELAY,dummy,STREAMING_TYPE,dummy,SOURCE_VIDEO_QUALITY,TETHERING_FLAG,CARRIER_ID,LAYER1ID,LAYER2ID,dummy,LAYER3ID,dummy,LAYER4ID,dummy,LAYER5ID,dummy,LAYER6ID,CHARGING_RULE_BASE_NAME,SP,dummy,EXTENDED_URL,SV,FIRST_SAI_CGI_ECGI,dummy,EXTENDED_URL_OTHER,SIGNALING_USE_FLAG,dummy,DNS_RETRANS_NUM,dummy,DNS_FAIL_CODE,FIRST_RAT,FIRST_RAT,MS_INDICATOR,LAST_SAI_CGI_ECGI,LAST_RAT,dummy,FIRST_LONGITUDE,dummy,FIRST_LATITUDE,dummy,FIRST_ALTITUDE,dummy,FIRST_RASTERLONGITUDE,dummy,FIRST_RASTERLATITUDE,dummy,FIRST_RASTERALTITUDE,dummy,FIRST_FREQUENCYSPOT,dummy,FIRST_CLUTTER,dummy,FIRST_USERBEHAVIOR,dummy,FIRST_SPEED,dummy,FIRST_CREDIBILITY,dummy,LAST_LONGITUDE,dummy,LAST_LATITUDE,dummy,LAST_ALTITUDE,dummy,LAST_RASTERLONGITUDE,dummy,LAST_RASTERLATITUDE,dummy,LAST_RASTERALTITUDE,dummy,LAST_FREQUENCYSPOT,dummy,LAST_CLUTTER,dummy,LAST_USERBEHAVIOR,dummy,LAST_SPEED,dummy,LAST_CREDIBILITY,dummy,IMEI_CIPHERTEXT,APP_ID,dummy,DOMAIN_NAME,dummy,STREAMING_CACHE_IP,dummy,STOP_LONGER_THAN_MIN_THRESHOLD,dummy,STOP_LONGER_THAN_MAX_THRESHOLD,dummy,PLAY_END_STAT,dummy,STOP_START_TIME1,dummy,STOP_END_TIME1,dummy,STOP_START_TIME2,dummy,STOP_END_TIME2,dummy,STOP_START_TIME3,dummy,STOP_END_TIME3,dummy,STOP_START_TIME4,dummy,STOP_END_TIME4,dummy,STOP_START_TIME5,dummy,STOP_END_TIME5,dummy,STOP_START_TIME6,dummy,STOP_END_TIME6,dummy,STOP_START_TIME7,dummy,STOP_END_TIME7,dummy,STOP_START_TIME8,dummy,STOP_END_TIME8,dummy,STOP_START_TIME9,dummy,STOP_END_TIME9,dummy,STOP_START_TIME10,dummy,STOP_END_TIME10,dummy,FAIL_CLASS,RECORD_TYPE,dummy,NODATA_COUNT,dummy,VIDEO_NODATA_DURATION,dummy,VIDEO_SMOOTH_DURATION,dummy,VIDEO_SD_DURATION,dummy,VIDEO_HD_DURATION,dummy,VIDEO_UHD_DURATION,dummy,VIDEO_FHD_DURATION,dummy,FLUCTUATION,dummy,START_DOWNLOAD_THROUGHPUT,dummy,L7_UL_GOODPUT_FULL_MSS,dummy,SESSIONKEY,dummy,FIRST_UCELLID,dummy,LAST_UCELLID,dummy,UCELLID1,dummy,LONGITUDE1,dummy,LATITUDE1,dummy,UCELLID2,dummy,LONGITUDE2,dummy,LATITUDE2,dummy,UCELLID3,dummy,LONGITUDE3,dummy,LATITUDE3,dummy,UCELLID4,dummy,LONGITUDE4,dummy,LATITUDE4,dummy,UCELLID5,dummy,LONGITUDE5,dummy,LATITUDE5,dummy,UCELLID6,dummy,LONGITUDE6,dummy,LATITUDE6,dummy,UCELLID7,dummy,LONGITUDE7,dummy,LATITUDE7,dummy,UCELLID8,dummy,LONGITUDE8,dummy,LATITUDE8,dummy,UCELLID9,dummy,LONGITUDE9,dummy,LATITUDE9,dummy,UCELLID10,dummy,LONGITUDE10,dummy,LATITUDE10,dummy,INTBUFFER_FULL_DELAY,dummy,STALL_DURATION,dummy,STREAMING_DW_PACKETS,dummy,STREAMING_DOWNLOAD_DELAY,dummy,PLAY_DURATION,dummy,STREAMING_QUALITY,dummy,VIDEO_DATA_RATE,dummy,AUDIO_DATA_RATE,dummy,STREAMING_FILESIZE,dummy,STREAMING_DURATIOIN,dummy,TCP_SYN_TIME,dummy,TCP_RTT_STEP1,CHARGE_ID,dummy,UL_REVERSE_TO_DL_DELAY,dummy,DL_REVERSE_TO_UL_DELAY,dummy,DATATRANS_DW_GOODPUT,dummy,DATATRANS_DW_TOTAL_DURATION,dummy,SUM_FRAGMENT_INTERVAL,dummy,TCP_FIN_TIMES,dummy,TCP_RESET_TIMES,dummy,URL_CLASSIFICATION,dummy,STREAMING_LQ_DURATIOIN,dummy,MAX_DNS_DELAY,dummy,MAX_DNS2SYN,dummy,MAX_LATANCY_OF_LINK_SETUP,dummy,MAX_SYNACK2FIRSTACK,dummy,MAX_SYNACK2LASTACK,dummy,MAX_ACK2GET_DELAY,dummy,MAX_FRAG_INTERVAL_PREDELAY,dummy,SUM_FRAG_INTERVAL_PREDELAY,dummy,SERVICE_DELAY_MSEC,dummy,HOMEPROVINCE,dummy,HOMECITY,dummy,SERVICE_ID,dummy,CHARGING_CLASS,dummy,DATATRANS_UL_DURATION,dummy,ASSOCIATED_ID,dummy,PACKET_LOSS_NUM,dummy,JITTER,dummy,MS_DNS_DELAY_MSEL,dummy,GET_STREAMING_DELAY,dummy,TCP_UL_RETRANS_WITHOUTPL,dummy,TCP_DW_RETRANS_WITHOUTPL,dummy,GET_MAX_UL_SIZE,dummy,GET_MIN_UL_SIZE,dummy,GET_MAX_DL_SIZE,dummy,GET_MIN_DL_SIZE,dummy,FLOW_SAMPLE_RATIO,dummy,UL_RTT_LONG_NUM,dummy,DW_RTT_LONG_NUM,dummy,UL_RTT_STAT_NUM,dummy,DW_RTT_STAT_NUM,dummy,USER_PROBE_UL_LOST_PKT,dummy,SERVER_PROBE_UL_LOST_PKT,dummy,SERVER_PROBE_DW_LOST_PKT,dummy,USER_PROBE_DW_LOST_PKT,dummy,AVG_DW_RTT_MICRO_SEC,dummy,AVG_UL_RTT_MICRO_SEC,dummy,RAN_NE_ID,dummy,AVG_UL_RTT,dummy,AVG_DW_RTT,dummy,CHARGING_CHARACTERISTICS,dummy,DL_SERIOUS_OUT_OF_ORDER_NUM,dummy,DL_SLIGHT_OUT_OF_ORDER_NUM,dummy,DL_FLIGHT_TOTAL_SIZE,dummy,DL_FLIGHT_TOTAL_NUM,dummy,DL_MAX_FLIGHT_SIZE,dummy,VALID_TRANS_DURATION,dummy,AIR_PORT_DURATION,dummy,RADIO_CONN_TIMES,dummy,UL_SERIOUS_OUT_OF_ORDER_NUM,dummy,UL_SLIGHT_OUT_OF_ORDER_NUM,dummy,UL_FLIGHT_TOTAL_SIZE,dummy,UL_FLIGHT_TOTAL_NUM,dummy,UL_MAX_FLIGHT_SIZE,dummy,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS,dummy,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS,dummy,DL_CONTINUOUS_RETRANSMISSION_DELAY,dummy,USER_HUNGRY_DELAY,dummy,SERVER_HUNGRY_DELAY,dummy,UPPERLAYER_IP_UL_FRAGMENTS,dummy,UPPERLAYER_IP_DL_FRAGMENTS,dummy,DOWNLAYER_IP_UL_FRAGMENTS,dummy,DOWNLAYER_IP_DL_FRAGMENTS,dummy,UPPERLAYER_IP_UL_PACKETS,dummy,UPPERLAYER_IP_DL_PACKETS,dummy,DOWNLAYER_IP_UL_PACKETS,dummy,DOWNLAYER_IP_DL_PACKETS,dummy,TCP_UL_PACKAGES_WITHPL,dummy,TCP_DW_PACKAGES_WITHPL,dummy,TCP_UL_PACKAGES_WITHOUTPL,dummy,TCP_DW_PACKAGES_WITHOUTPL,dummy,TCP_UL_RETRANS_WITHPL,dummy,TCP_DW_RETRANS_WITHPL,L4_UL_THROUGHPUT,L4_DW_THROUGHPUT,L4_UL_GOODPUT,L4_DW_GOODPUT,NETWORK_UL_TRAFFIC,NETWORK_DL_TRAFFIC,L4_UL_PACKETS,L4_DW_PACKETS,TCP_RTT,TCP_UL_OUTOFSEQU,TCP_DW_OUTOFSEQU,TCP_UL_RETRANS,TCP_DW_RETRANS')""").collect
-
-    sql(s"""LOAD DATA INPATH '$resourcesPath/Data/SEQ500/seq_500Records_hive2.csv' into table smart_500_DINC_hive """).collect
-
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC001
-  test("Query_DICTIONARY_INCLUDE_TC001", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DINC where HOST not in ('www.hua735435.com')""",
-      s"""select SID, IMEI from smart_500_DINC_hive where HOST not in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC001")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC002
-  test("Query_DICTIONARY_INCLUDE_TC002", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DINC where HOST in  ('www.hua735435.com')""",
-      s"""select SID, IMEI from smart_500_DINC_hive where HOST in  ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC002")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC003
-  test("Query_DICTIONARY_INCLUDE_TC003", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DINC where HOST LIKE  'www.hua735435.com'""",
-      s"""select SID, IMEI from smart_500_DINC_hive where HOST LIKE  'www.hua735435.com'""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC003")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC004
-  test("Query_DICTIONARY_INCLUDE_TC004", Include) {
-
-    checkAnswer(s"""select SID, IMEI from smart_500_DINC where HOST Not LIKE  'www.hua735435.com'""",
-      s"""select SID, IMEI from smart_500_DINC_hive where HOST Not LIKE  'www.hua735435.com'""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC004")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC005
-  test("Query_DICTIONARY_INCLUDE_TC005", Include) {
-
-    checkAnswer(s"""select length(HOST) from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST) from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC005")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC006
-  test("Query_DICTIONARY_INCLUDE_TC006", Include) {
-
-    checkAnswer(s"""select avg(HOST),avg(LAYER1ID) from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select avg(HOST),avg(LAYER1ID) from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC006")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC007
-  test("Query_DICTIONARY_INCLUDE_TC007", Include) {
-
-    checkAnswer(s"""select avg(HOST),avg(LAYER1ID) from smart_500_DINC where HOST not in ('www.hua735435.com')""",
-      s"""select avg(HOST),avg(LAYER1ID) from smart_500_DINC_hive where HOST not in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC007")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC008
-  test("Query_DICTIONARY_INCLUDE_TC008", Include) {
-
-    checkAnswer(s"""select substring(IMEI,1,4) from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select substring(IMEI,1,4) from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC008")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC009
-  test("Query_DICTIONARY_INCLUDE_TC009", Include) {
-
-    checkAnswer(s"""select length(HOST)+10 from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)+10 from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC009")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC010
-  test("Query_DICTIONARY_INCLUDE_TC010", Include) {
-
-    checkAnswer(s"""select length(HOST)-10 from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)-10 from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC010")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC011
-  test("Query_DICTIONARY_INCLUDE_TC011", Include) {
-
-    checkAnswer(s"""select length(HOST)/10 from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)/10 from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC011")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC012
-  test("Query_DICTIONARY_INCLUDE_TC012", Include) {
-
-    checkAnswer(s"""select length(HOST)*10 from smart_500_DINC where HOST in ('www.hua735435.com')""",
-      s"""select length(HOST)*10 from smart_500_DINC_hive where HOST in ('www.hua735435.com')""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC012")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC013
-  test("Query_DICTIONARY_INCLUDE_TC013", Include) {
-
-    checkAnswer(s"""select lower(MS_IP),sum(LAYER1ID) from smart_500_DINC  group by lower(MS_IP)""",
-      s"""select lower(MS_IP),sum(LAYER1ID) from smart_500_DINC_hive  group by lower(MS_IP)""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC013")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC014
-  test("Query_DICTIONARY_INCLUDE_TC014", Include) {
-
-    checkAnswer(s"""select * from smart_500_DINC  where unix_timestamp(MS_IP)=1420268400""",
-      s"""select * from smart_500_DINC_hive  where unix_timestamp(MS_IP)=1420268400""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC014")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC015
-  test("Query_DICTIONARY_INCLUDE_TC015", Include) {
-
-    checkAnswer(s"""select * from smart_500_DINC  where to_date(MS_IP)='2015-01-07'""",
-      s"""select * from smart_500_DINC_hive  where to_date(MS_IP)='2015-01-07'""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC015")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC016
-  test("Query_DICTIONARY_INCLUDE_TC016", Include) {
-
-    checkAnswer(s"""select * from smart_500_DINC  where datediff(MS_IP,'2014-12-01')>=35""",
-      s"""select * from smart_500_DINC_hive  where datediff(MS_IP,'2014-12-01')>=35""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC016")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC017
-  test("Query_DICTIONARY_INCLUDE_TC017", Include) {
-
-    checkAnswer(s"""select MS_IP,count(*) from smart_500_DINC  group by MS_IP""",
-      s"""select MS_IP,count(*) from smart_500_DINC_hive  group by MS_IP""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC017")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC018
-  test("Query_DICTIONARY_INCLUDE_TC018", Include) {
-
-    sql(s"""select MS_IP,SID,count(*) from smart_500_DINC  group by MS_IP,SID order by MS_IP limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC019
-  test("Query_DICTIONARY_INCLUDE_TC019", Include) {
-
-    sql(s"""select SID,length( MSISDN),avg(LAYER1ID),avg(TCP_DW_RETRANS) from smart_500_DINC  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC020
-  test("Query_DICTIONARY_INCLUDE_TC020", Include) {
-
-    sql(s"""select SID,length( MSISDN),max(LAYER1ID),min(LAYER1ID) from smart_500_DINC  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC021
-  test("Query_DICTIONARY_INCLUDE_TC021", Include) {
-
-    sql(s"""select SID,length( MSISDN),max(LAYER1ID),max(LAYER1ID) from smart_500_DINC  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC022
-  test("Query_DICTIONARY_INCLUDE_TC022", Include) {
-
-    sql(s"""select SID,length( MSISDN),min(LAYER1ID),min(LAYER1ID) from smart_500_DINC  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC023
-  test("Query_DICTIONARY_INCLUDE_TC023", Include) {
-
-    sql(s"""select SID,length( MSISDN),max(LAYER1ID),min(LAYER1ID),avg(LAYER1ID) from smart_500_DINC  group by SID,length( MSISDN) order by SID limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC024
-  test("Query_DICTIONARY_INCLUDE_TC024", Include) {
-
-    checkAnswer(s"""select concat(upper(MSISDN),1),sum(LAYER1ID) from smart_500_DINC  group by concat(upper(MSISDN),1)""",
-      s"""select concat(upper(MSISDN),1),sum(LAYER1ID) from smart_500_DINC_hive  group by concat(upper(MSISDN),1)""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC024")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC025
-  test("Query_DICTIONARY_INCLUDE_TC025", Include) {
-
-    checkAnswer(s"""select upper(substring(MSISDN,1,4)),sum(LAYER1ID) from smart_500_DINC group by upper(substring(MSISDN,1,4)) """,
-      s"""select upper(substring(MSISDN,1,4)),sum(LAYER1ID) from smart_500_DINC_hive group by upper(substring(MSISDN,1,4)) """, "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC025")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC026
-  test("Query_DICTIONARY_INCLUDE_TC026", Include) {
-
-    checkAnswer(s"""select max(SERVER_IP) from smart_500_DINC""",
-      s"""select max(SERVER_IP) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC026")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC027
-  test("Query_DICTIONARY_INCLUDE_TC027", Include) {
-
-    checkAnswer(s"""select max(SERVER_IP+10) from smart_500_DINC""",
-      s"""select max(SERVER_IP+10) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC027")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC028
-  test("Query_DICTIONARY_INCLUDE_TC028", Include) {
-
-    checkAnswer(s"""select max(MSISDN) from smart_500_DINC""",
-      s"""select max(MSISDN) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC028")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC029
-  test("Query_DICTIONARY_INCLUDE_TC029", Include) {
-
-    checkAnswer(s"""select max(MSISDN+10) from smart_500_DINC""",
-      s"""select max(MSISDN+10) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC029")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC030
-  test("Query_DICTIONARY_INCLUDE_TC030", Include) {
-
-    checkAnswer(s"""select avg(TCP_DW_RETRANS) from smart_500_DINC""",
-      s"""select avg(TCP_DW_RETRANS) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC030")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC031
-  test("Query_DICTIONARY_INCLUDE_TC031", Include) {
-
-    checkAnswer(s"""select avg(TCP_DW_RETRANS+10) from smart_500_DINC""",
-      s"""select avg(TCP_DW_RETRANS+10) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC031")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC032
-  test("Query_DICTIONARY_INCLUDE_TC032", Include) {
-
-    checkAnswer(s"""select avg(TCP_DW_RETRANS-10) from smart_500_DINC""",
-      s"""select avg(TCP_DW_RETRANS-10) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC032")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC033
-  test("Query_DICTIONARY_INCLUDE_TC033", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS)-10 from smart_500_DINC""",
-      s"""select count(TCP_DW_RETRANS)-10 from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC033")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC034
-  test("Query_DICTIONARY_INCLUDE_TC034", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS)-10 from smart_500_DINC""",
-      s"""select count(TCP_DW_RETRANS)-10 from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC034")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC035
-  test("Query_DICTIONARY_INCLUDE_TC035", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS)-10 from smart_500_DINC""",
-      s"""select count(TCP_DW_RETRANS)-10 from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC035")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC036
-  test("Query_DICTIONARY_INCLUDE_TC036", Include) {
-
-    checkAnswer(s"""select sum(MSISDN), sum(DISTINCT MSISDN) from smart_500_DINC""",
-      s"""select sum(MSISDN), sum(DISTINCT MSISDN) from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC036")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC037
-  test("Query_DICTIONARY_INCLUDE_TC037", Include) {
-
-    checkAnswer(s"""select count (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC""",
-      s"""select count (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC037")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC038
-  test("Query_DICTIONARY_INCLUDE_TC038", Include) {
-
-    checkAnswer(s"""select count (if(TCP_DW_RETRANS<100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC""",
-      s"""select count (if(TCP_DW_RETRANS<100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC038")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC039
-  test("Query_DICTIONARY_INCLUDE_TC039", Include) {
-
-    checkAnswer(s"""select count (if(TCP_DW_RETRANS=100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC""",
-      s"""select count (if(TCP_DW_RETRANS=100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC039")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC040
-  test("Query_DICTIONARY_INCLUDE_TC040", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS) from smart_500_DINC where TCP_DW_RETRANS=100""",
-      s"""select count(TCP_DW_RETRANS) from smart_500_DINC_hive where TCP_DW_RETRANS=100""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC040")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC041
-  test("Query_DICTIONARY_INCLUDE_TC041", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS) from smart_500_DINC where TCP_DW_RETRANS<100""",
-      s"""select count(TCP_DW_RETRANS) from smart_500_DINC_hive where TCP_DW_RETRANS<100""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC041")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC042
-  test("Query_DICTIONARY_INCLUDE_TC042", Include) {
-
-    checkAnswer(s"""select count(TCP_DW_RETRANS) from smart_500_DINC where TCP_DW_RETRANS>100""",
-      s"""select count(TCP_DW_RETRANS) from smart_500_DINC_hive where TCP_DW_RETRANS>100""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC042")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC043
-  test("Query_DICTIONARY_INCLUDE_TC043", Include) {
-
-    sql(s"""select MSISDN, TCP_DW_RETRANS + LAYER1ID as a  from smart_500_DINC order by MSISDN limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC044
-  test("Query_DICTIONARY_INCLUDE_TC044", Include) {
-
-    sql(s"""select MSISDN, sum(TCP_DW_RETRANS + 10) Total from smart_500_DINC group by  MSISDN order by Total limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC045
-  test("Query_DICTIONARY_INCLUDE_TC045", Include) {
-
-    checkAnswer(s"""select MSISDN, min(LAYER1ID + 10)  Total from smart_500_DINC group by  MSISDN order by MSISDN""",
-      s"""select MSISDN, min(LAYER1ID + 10)  Total from smart_500_DINC_hive group by  MSISDN order by MSISDN""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC045")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC046
-  test("Query_DICTIONARY_INCLUDE_TC046", Include) {
-
-    checkAnswer(s"""select avg (if(LAYER1ID>100,NULL,LAYER1ID))  a from smart_500_DINC""",
-      s"""select avg (if(LAYER1ID>100,NULL,LAYER1ID))  a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC046")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC047
-  test("Query_DICTIONARY_INCLUDE_TC047", Include) {
-
-    checkAnswer(s"""select avg (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC""",
-      s"""select avg (if(TCP_DW_RETRANS>100,NULL,TCP_DW_RETRANS))  a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC047")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC048
-  ignore("Query_DICTIONARY_INCLUDE_TC048", Include) {
-
-    checkAnswer(s"""select variance(LAYER1ID) as a   from smart_500_DINC""",
-      s"""select variance(LAYER1ID) as a   from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC048")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC049
-  ignore("Query_DICTIONARY_INCLUDE_TC049", Include) {
-
-    checkAnswer(s"""select var_pop(LAYER1ID)  as a from smart_500_DINC""",
-      s"""select var_pop(LAYER1ID)  as a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC049")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC050
-  ignore("Query_DICTIONARY_INCLUDE_TC050", Include) {
-
-    checkAnswer(s"""select var_samp(LAYER1ID) as a  from smart_500_DINC""",
-      s"""select var_samp(LAYER1ID) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC050")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC051
-  ignore("Query_DICTIONARY_INCLUDE_TC051", Include) {
-
-    checkAnswer(s"""select stddev_pop(LAYER1ID) as a  from smart_500_DINC""",
-      s"""select stddev_pop(LAYER1ID) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC051")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC052
-  ignore("Query_DICTIONARY_INCLUDE_TC052", Include) {
-
-    checkAnswer(s"""select stddev_samp(LAYER1ID)  as a from smart_500_DINC""",
-      s"""select stddev_samp(LAYER1ID)  as a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC052")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC053
-  ignore("Query_DICTIONARY_INCLUDE_TC053", Include) {
-
-    checkAnswer(s"""select covar_pop(LAYER1ID,LAYER1ID) as a  from smart_500_DINC""",
-      s"""select covar_pop(LAYER1ID,LAYER1ID) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC053")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC054
-  ignore("Query_DICTIONARY_INCLUDE_TC054", Include) {
-
-    checkAnswer(s"""select covar_samp(LAYER1ID,LAYER1ID) as a  from smart_500_DINC""",
-      s"""select covar_samp(LAYER1ID,LAYER1ID) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC054")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC055
-  test("Query_DICTIONARY_INCLUDE_TC055", Include) {
-
-    checkAnswer(s"""select corr(LAYER1ID,LAYER1ID)  as a from smart_500_DINC""",
-      s"""select corr(LAYER1ID,LAYER1ID)  as a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC055")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC056
-  test("Query_DICTIONARY_INCLUDE_TC056", Include) {
-
-    checkAnswer(s"""select percentile(LAYER1ID,0.2) as  a  from smart_500_DINC""",
-      s"""select percentile(LAYER1ID,0.2) as  a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC056")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC057
-  test("Query_DICTIONARY_INCLUDE_TC057", Include) {
-
-    checkAnswer(s"""select percentile(LAYER1ID,array(0,0.2,0.3,1))  as  a from smart_500_DINC""",
-      s"""select percentile(LAYER1ID,array(0,0.2,0.3,1))  as  a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC057")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC058
-  test("Query_DICTIONARY_INCLUDE_TC058", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,0.2) as a  from (select LAYER1ID from smart_500_DINC order by LAYER1ID) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC059
-  test("Query_DICTIONARY_INCLUDE_TC059", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,0.2,5) as a  from (select LAYER1ID from smart_500_DINC order by LAYER1ID) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC060
-  test("Query_DICTIONARY_INCLUDE_TC060", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,array(0.2,0.3,0.99))  as a from (select LAYER1ID from smart_500_DINC order by LAYER1ID) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC061
-  test("Query_DICTIONARY_INCLUDE_TC061", Include) {
-
-    sql(s"""select percentile_approx(LAYER1ID,array(0.2,0.3,0.99),5) as a from (select LAYER1ID from smart_500_DINC order by LAYER1ID) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC062
-  test("Query_DICTIONARY_INCLUDE_TC062", Include) {
-
-    sql(s"""select histogram_numeric(LAYER1ID,2)  as a from (select LAYER1ID from smart_500_DINC order by LAYER1ID) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC063
-  ignore("Query_DICTIONARY_INCLUDE_TC063", Include) {
-
-    checkAnswer(s"""select variance(TCP_DW_RETRANS) as a   from smart_500_DINC""",
-      s"""select variance(TCP_DW_RETRANS) as a   from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC063")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC064
-  ignore("Query_DICTIONARY_INCLUDE_TC064", Include) {
-
-    checkAnswer(s"""select var_pop(TCP_DW_RETRANS)  as a from smart_500_DINC""",
-      s"""select var_pop(TCP_DW_RETRANS)  as a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC064")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC065
-  ignore("Query_DICTIONARY_INCLUDE_TC065", Include) {
-
-    checkAnswer(s"""select var_samp(TCP_DW_RETRANS) as a  from smart_500_DINC""",
-      s"""select var_samp(TCP_DW_RETRANS) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC065")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC066
-  ignore("Query_DICTIONARY_INCLUDE_TC066", Include) {
-
-    checkAnswer(s"""select stddev_pop(TCP_DW_RETRANS) as a  from smart_500_DINC""",
-      s"""select stddev_pop(TCP_DW_RETRANS) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC066")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC067
-  ignore("Query_DICTIONARY_INCLUDE_TC067", Include) {
-
-    checkAnswer(s"""select stddev_samp(TCP_DW_RETRANS)  as a from smart_500_DINC""",
-      s"""select stddev_samp(TCP_DW_RETRANS)  as a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC067")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC068
-  ignore("Query_DICTIONARY_INCLUDE_TC068", Include) {
-
-    checkAnswer(s"""select covar_pop(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DINC""",
-      s"""select covar_pop(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC068")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC069
-  ignore("Query_DICTIONARY_INCLUDE_TC069", Include) {
-
-    checkAnswer(s"""select covar_samp(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DINC""",
-      s"""select covar_samp(TCP_DW_RETRANS,TCP_DW_RETRANS) as a  from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC069")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC070
-  test("Query_DICTIONARY_INCLUDE_TC070", Include) {
-
-    checkAnswer(s"""select corr(TCP_DW_RETRANS,TCP_DW_RETRANS)  as a from smart_500_DINC""",
-      s"""select corr(TCP_DW_RETRANS,TCP_DW_RETRANS)  as a from smart_500_DINC_hive""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC070")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC073
-  test("Query_DICTIONARY_INCLUDE_TC073", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,0.2) as a  from (select TCP_DW_RETRANS from smart_500_DINC order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC074
-  test("Query_DICTIONARY_INCLUDE_TC074", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,0.2,5) as a  from (select TCP_DW_RETRANS from smart_500_DINC order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC075
-  test("Query_DICTIONARY_INCLUDE_TC075", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,array(0.2,0.3,0.99))  as a from (select TCP_DW_RETRANS from smart_500_DINC order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC076
-  test("Query_DICTIONARY_INCLUDE_TC076", Include) {
-
-    sql(s"""select percentile_approx(TCP_DW_RETRANS,array(0.2,0.3,0.99),5) as a from (select TCP_DW_RETRANS from smart_500_DINC order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC077
-  test("Query_DICTIONARY_INCLUDE_TC077", Include) {
-
-    sql(s"""select histogram_numeric(TCP_DW_RETRANS,2)  as a from (select TCP_DW_RETRANS from smart_500_DINC order by TCP_DW_RETRANS) t""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC094
-  test("Query_DICTIONARY_INCLUDE_TC094", Include) {
-
-    sql(s"""select Upper(streaming_url) a ,host from smart_500_DINC order by host limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC095
-  test("Query_DICTIONARY_INCLUDE_TC095", Include) {
-
-    sql(s"""select Lower(streaming_url) a  from smart_500_DINC order by host limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC096
-  test("Query_DICTIONARY_INCLUDE_TC096", Include) {
-
-    sql(s"""select streaming_url as b,LAYER1ID as a from smart_500_DINC  order by a,b asc limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC097
-  test("Query_DICTIONARY_INCLUDE_TC097", Include) {
-
-    sql(s"""select streaming_url as b,TCP_DW_RETRANS as a from smart_500_DINC  order by a,b desc limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC098
-  test("Query_DICTIONARY_INCLUDE_TC098", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where streaming_url ='www.hua1/xyz'""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where streaming_url ='www.hua1/xyz'""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC098")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC099
-  test("Query_DICTIONARY_INCLUDE_TC099", Include) {
-
-    sql(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where streaming_url ='www.hua90/xyz' and TCP_DW_RETRANS ='82.0' limit 10""").collect
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC100
-  test("Query_DICTIONARY_INCLUDE_TC100", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where streaming_url ='www.hua1/xyz' or  TCP_DW_RETRANS ='82.0'""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where streaming_url ='www.hua1/xyz' or  TCP_DW_RETRANS ='82.0'""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC100")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC101
-  test("Query_DICTIONARY_INCLUDE_TC101", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where streaming_url !='www.hua1/xyz'  order by LAYER1ID,TCP_DW_RETRANS,streaming_url""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where streaming_url !='www.hua1/xyz'  order by LAYER1ID,TCP_DW_RETRANS,streaming_url""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC101")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC102
-  test("Query_DICTIONARY_INCLUDE_TC102", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where streaming_url !='www.hua1/xyz' and TCP_DW_RETRANS !='152.0'  order by  LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where streaming_url !='www.hua1/xyz' and TCP_DW_RETRANS !='152.0'  order by  LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC102")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC103
-  test("Query_DICTIONARY_INCLUDE_TC103", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where TCP_DW_RETRANS >2.0 order by LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where TCP_DW_RETRANS >2.0 order by LAYER1ID,TCP_DW_RETRANS,streaming_url limit 10""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC103")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC104
-  test("Query_DICTIONARY_INCLUDE_TC104", Include) {
-
-    checkAnswer(s"""select LAYER1ID as a from smart_500_DINC where LAYER1ID<=>LAYER1ID order by LAYER1ID desc limit 10""",
-      s"""select LAYER1ID as a from smart_500_DINC_hive where LAYER1ID<=>LAYER1ID order by LAYER1ID desc limit 10""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC104")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC105
-  test("Query_DICTIONARY_INCLUDE_TC105", Include) {
-
-    checkAnswer(s"""SELECT LAYER1ID,TCP_DW_RETRANS,streaming_url FROM (select * from smart_500_DINC) SUB_QRY ORDER BY LAYER1ID,TCP_DW_RETRANS,streaming_url ASC limit 10""",
-      s"""SELECT LAYER1ID,TCP_DW_RETRANS,streaming_url FROM (select * from smart_500_DINC_hive) SUB_QRY ORDER BY LAYER1ID,TCP_DW_RETRANS,streaming_url ASC limit 10""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC105")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC106
-  test("Query_DICTIONARY_INCLUDE_TC106", Include) {
-
-    checkAnswer(s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where ( LAYER1ID+1) == 101 order by TCP_DW_RETRANS,LAYER1ID limit 5""",
-      s"""select LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where ( LAYER1ID+1) == 101 order by TCP_DW_RETRANS,LAYER1ID limit 5""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC106")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC107
-  test("Query_DICTIONARY_INCLUDE_TC107", Include) {
-
-    checkAnswer(s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where  streaming_url is  null order by LAYER1ID,TCP_DW_RETRANS,streaming_url                                """,
-      s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where  streaming_url is  null order by LAYER1ID,TCP_DW_RETRANS,streaming_url                                """, "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC107")
-
-  }
-
-
-  //Query_DICTIONARY_INCLUDE_TC108
-  test("Query_DICTIONARY_INCLUDE_TC108", Include) {
-
-    checkAnswer(s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC where  streaming_url is  not null order by LAYER1ID,TCP_DW_RETRANS,streaming_url""",
-      s"""select  LAYER1ID,TCP_DW_RETRANS,streaming_url from smart_500_DINC_hive where  streaming_url is  not null order by LAYER1ID,TCP_DW_RETRANS,streaming_url""", "QueriesIncludeDictionaryTestCase_Query_DICTIONARY_INCLUDE_TC108")
-
-  }
-
-  override def afterAll {
-    sql("drop table if exists TABLE_DICTIONARY_INCLUDE")
-    sql("drop table if exists TABLE_DICTIONARY_INCLUDE_hive")
-    sql("drop table if exists TABLE_DICTIONARY_INCLUDE1")
-    sql("drop table if exists TABLE_DICTIONARY_INCLUDE1_hive")
-    sql("drop table if exists smart_500_DINC")
-    sql("drop table if exists smart_500_DINC_hive")
-  }
-}
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesNormalTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesNormalTestCase.scala
index 0411dcb..049b975 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesNormalTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesNormalTestCase.scala
@@ -31,7 +31,7 @@
   test("OSCON_Query_CreateTable", Include) {
     sql(s"""drop table if exists oscon_carbon""").collect
 
-    sql(s"""CREATE TABLE oscon_carbon (CUST_ID String,CUST_COUNTRY String,CUST_STATE String,CUST_CITY String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,WH_COUNTRY String,WH_STATE String,WH_CITY String,WH_COUNTY String,CUST_JOB_TITLE String,CUST_BUY_POTENTIAL String,PROD_UNQ_MDL_ID String,PROD_BRAND_NAME String,PRODUCT_NAME String,PRODUCT_MODEL String,PROD_MODEL_ID String,PROD_STD_CST int,REG_UNIT_PRICE int,TOTAL_PRD_COST int,TOTAL_PRD_DISC int,PROD_OFF_PRICE int,TOTAL_TX_AMT int,PROD_UNITS int,PROD_WEIGHT int,UNIT_PRICE_DSCNT_PCT int,DSCNT_AMT int,OL_SALES_PRICE int,PROD_COLOR String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,TRACKING_NO String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_NO String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_NICK_NAME String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_PRFRD_FLG String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_LOGIN String,CUST_EMAIL_ADDR String,CUST_LAST_RVW_DATE String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_BAR_CODE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_ZIP String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,CUST_DEP_COUNT int,CUST_VEHICLE_COUNT int,CUST_ADDRESS_CNT int,CUST_CRNT_CDEMO_CNT int,CUST_CRNT_HDEMO_CNT int,CUST_CRNT_ADDR_DM int,CUST_FIRST_SHIPTO_CNT int,CUST_FIRST_SALES_CNT int,CUST_GMT_OFFSET int,CUST_DEMO_CNT int,CUST_INCOME int,PROD_UNLIMITED int,EXTENDED_AMT int,FREIGHT_CHRG int,WAITING_PERIOD int,DELIVERY_PERIOD int,ITM_CRNT_PRICE int,ITM_UNITS int,ITM_WSLE_CST int,ITM_SIZE int,PRM_CST int,PRM_RESPONSE_TARGET int,PRM_ITM_DM int,SHP_MODE_CNT int,WH_GMT_OFFSET int,WH_SQ_FT int,STR_ORD_QTY int,STR_WSLE_CST int,STR_LIST_PRICE int,STR_SALES_PRICE int,STR_EXT_DSCNT_AMT int,STR_EXT_SALES_PRICE int,STR_EXT_WSLE_CST int,STR_EXT_LIST_PRICE int,STR_EXT_TX int,STR_COUPON_AMT int,STR_NET_PAID int,STR_NET_PAID_INC_TX int,STR_NET_PRFT int,STR_SOLD_YR_CNT int,STR_SOLD_MM_CNT int,STR_SOLD_ITM_CNT int,STR_TOTAL_CUST_CNT int,STR_AREA_CNT int,STR_DEMO_CNT int,STR_OFFER_CNT int,STR_PRM_CNT int,STR_TICKET_CNT int,STR_NET_PRFT_DM_A int,STR_NET_PRFT_DM_B int,STR_NET_PRFT_DM_C int,STR_NET_PRFT_DM_D int,STR_NET_PRFT_DM_E int,STR_RET_STR_ID int,STR_RET_REASON_CNT int,STR_RET_TICKET_NO int,STR_RTRN_QTY int,STR_RTRN_AMT int,STR_RTRN_TX int,STR_RTRN_AMT_INC_TX int,STR_RET_FEE int,STR_RTRN_SHIP_CST int,STR_RFNDD_CSH int,STR_REVERSED_CHRG int,STR_STR_CREDIT int,STR_RET_NET_LOSS int,STR_RTRNED_YR_CNT int,STR_RTRN_MM_CNT int,STR_RET_ITM_CNT int,STR_RET_CUST_CNT int,STR_RET_AREA_CNT int,STR_RET_OFFER_CNT int,STR_RET_PRM_CNT int,STR_RET_NET_LOSS_DM_A int,STR_RET_NET_LOSS_DM_B int,STR_RET_NET_LOSS_DM_C int,STR_RET_NET_LOSS_DM_D int,OL_ORD_QTY int,OL_WSLE_CST int,OL_LIST_PRICE int,OL_EXT_DSCNT_AMT int,OL_EXT_SALES_PRICE int,OL_EXT_WSLE_CST int,OL_EXT_LIST_PRICE int,OL_EXT_TX int,OL_COUPON_AMT int,OL_EXT_SHIP_CST int,OL_NET_PAID int,OL_NET_PAID_INC_TX int,OL_NET_PAID_INC_SHIP int,OL_NET_PAID_INC_SHIP_TX int,OL_NET_PRFT int,OL_SOLD_YR_CNT int,OL_SOLD_MM_CNT int,OL_SHIP_DATE_CNT int,OL_ITM_CNT int,OL_BILL_CUST_CNT int,OL_BILL_AREA_CNT int,OL_BILL_DEMO_CNT int,OL_BILL_OFFER_CNT int,OL_SHIP_CUST_CNT int,OL_SHIP_AREA_CNT int,OL_SHIP_DEMO_CNT int,OL_SHIP_OFFER_CNT int,OL_WEB_PAGE_CNT int,OL_WEB_SITE_CNT int,OL_SHIP_MODE_CNT int,OL_WH_CNT int,OL_PRM_CNT int,OL_NET_PRFT_DM_A int,OL_NET_PRFT_DM_B int,OL_NET_PRFT_DM_C int,OL_NET_PRFT_DM_D int,OL_RET_RTRN_QTY int,OL_RTRN_AMT int,OL_RTRN_TX int,OL_RTRN_AMT_INC_TX int,OL_RET_FEE int,OL_RTRN_SHIP_CST int,OL_RFNDD_CSH int,OL_REVERSED_CHRG int,OL_ACCOUNT_CREDIT int,OL_RTRNED_YR_CNT int,OL_RTRNED_MM_CNT int,OL_RTRITM_CNT int,OL_RFNDD_CUST_CNT int,OL_RFNDD_AREA_CNT int,OL_RFNDD_DEMO_CNT int,OL_RFNDD_OFFER_CNT int,OL_RTRNING_CUST_CNT int,OL_RTRNING_AREA_CNT int,OL_RTRNING_DEMO_CNT int,OL_RTRNING_OFFER_CNT int,OL_RTRWEB_PAGE_CNT int,OL_REASON_CNT int,OL_NET_LOSS int,OL_NET_LOSS_DM_A int,OL_NET_LOSS_DM_B int,OL_NET_LOSS_DM_C int)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='PROD_UNQ_MDL_ID,CUST_NICK_NAME,CUST_LOGIN,CUST_EMAIL_ADDR,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,PROD_BAR_CODE,TRACKING_NO,STR_ORDER_NO')""").collect
+    sql(s"""CREATE TABLE oscon_carbon (CUST_ID String,CUST_COUNTRY String,CUST_STATE String,CUST_CITY String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,WH_COUNTRY String,WH_STATE String,WH_CITY String,WH_COUNTY String,CUST_JOB_TITLE String,CUST_BUY_POTENTIAL String,PROD_UNQ_MDL_ID String,PROD_BRAND_NAME String,PRODUCT_NAME String,PRODUCT_MODEL String,PROD_MODEL_ID String,PROD_STD_CST int,REG_UNIT_PRICE int,TOTAL_PRD_COST int,TOTAL_PRD_DISC int,PROD_OFF_PRICE int,TOTAL_TX_AMT int,PROD_UNITS int,PROD_WEIGHT int,UNIT_PRICE_DSCNT_PCT int,DSCNT_AMT int,OL_SALES_PRICE int,PROD_COLOR String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,TRACKING_NO String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_NO String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_NICK_NAME String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_PRFRD_FLG String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_LOGIN String,CUST_EMAIL_ADDR String,CUST_LAST_RVW_DATE String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_BAR_CODE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_ZIP String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,CUST_DEP_COUNT int,CUST_VEHICLE_COUNT int,CUST_ADDRESS_CNT int,CUST_CRNT_CDEMO_CNT int,CUST_CRNT_HDEMO_CNT int,CUST_CRNT_ADDR_DM int,CUST_FIRST_SHIPTO_CNT int,CUST_FIRST_SALES_CNT int,CUST_GMT_OFFSET int,CUST_DEMO_CNT int,CUST_INCOME int,PROD_UNLIMITED int,EXTENDED_AMT int,FREIGHT_CHRG int,WAITING_PERIOD int,DELIVERY_PERIOD int,ITM_CRNT_PRICE int,ITM_UNITS int,ITM_WSLE_CST int,ITM_SIZE int,PRM_CST int,PRM_RESPONSE_TARGET int,PRM_ITM_DM int,SHP_MODE_CNT int,WH_GMT_OFFSET int,WH_SQ_FT int,STR_ORD_QTY int,STR_WSLE_CST int,STR_LIST_PRICE int,STR_SALES_PRICE int,STR_EXT_DSCNT_AMT int,STR_EXT_SALES_PRICE int,STR_EXT_WSLE_CST int,STR_EXT_LIST_PRICE int,STR_EXT_TX int,STR_COUPON_AMT int,STR_NET_PAID int,STR_NET_PAID_INC_TX int,STR_NET_PRFT int,STR_SOLD_YR_CNT int,STR_SOLD_MM_CNT int,STR_SOLD_ITM_CNT int,STR_TOTAL_CUST_CNT int,STR_AREA_CNT int,STR_DEMO_CNT int,STR_OFFER_CNT int,STR_PRM_CNT int,STR_TICKET_CNT int,STR_NET_PRFT_DM_A int,STR_NET_PRFT_DM_B int,STR_NET_PRFT_DM_C int,STR_NET_PRFT_DM_D int,STR_NET_PRFT_DM_E int,STR_RET_STR_ID int,STR_RET_REASON_CNT int,STR_RET_TICKET_NO int,STR_RTRN_QTY int,STR_RTRN_AMT int,STR_RTRN_TX int,STR_RTRN_AMT_INC_TX int,STR_RET_FEE int,STR_RTRN_SHIP_CST int,STR_RFNDD_CSH int,STR_REVERSED_CHRG int,STR_STR_CREDIT int,STR_RET_NET_LOSS int,STR_RTRNED_YR_CNT int,STR_RTRN_MM_CNT int,STR_RET_ITM_CNT int,STR_RET_CUST_CNT int,STR_RET_AREA_CNT int,STR_RET_OFFER_CNT int,STR_RET_PRM_CNT int,STR_RET_NET_LOSS_DM_A int,STR_RET_NET_LOSS_DM_B int,STR_RET_NET_LOSS_DM_C int,STR_RET_NET_LOSS_DM_D int,OL_ORD_QTY int,OL_WSLE_CST int,OL_LIST_PRICE int,OL_EXT_DSCNT_AMT int,OL_EXT_SALES_PRICE int,OL_EXT_WSLE_CST int,OL_EXT_LIST_PRICE int,OL_EXT_TX int,OL_COUPON_AMT int,OL_EXT_SHIP_CST int,OL_NET_PAID int,OL_NET_PAID_INC_TX int,OL_NET_PAID_INC_SHIP int,OL_NET_PAID_INC_SHIP_TX int,OL_NET_PRFT int,OL_SOLD_YR_CNT int,OL_SOLD_MM_CNT int,OL_SHIP_DATE_CNT int,OL_ITM_CNT int,OL_BILL_CUST_CNT int,OL_BILL_AREA_CNT int,OL_BILL_DEMO_CNT int,OL_BILL_OFFER_CNT int,OL_SHIP_CUST_CNT int,OL_SHIP_AREA_CNT int,OL_SHIP_DEMO_CNT int,OL_SHIP_OFFER_CNT int,OL_WEB_PAGE_CNT int,OL_WEB_SITE_CNT int,OL_SHIP_MODE_CNT int,OL_WH_CNT int,OL_PRM_CNT int,OL_NET_PRFT_DM_A int,OL_NET_PRFT_DM_B int,OL_NET_PRFT_DM_C int,OL_NET_PRFT_DM_D int,OL_RET_RTRN_QTY int,OL_RTRN_AMT int,OL_RTRN_TX int,OL_RTRN_AMT_INC_TX int,OL_RET_FEE int,OL_RTRN_SHIP_CST int,OL_RFNDD_CSH int,OL_REVERSED_CHRG int,OL_ACCOUNT_CREDIT int,OL_RTRNED_YR_CNT int,OL_RTRNED_MM_CNT int,OL_RTRITM_CNT int,OL_RFNDD_CUST_CNT int,OL_RFNDD_AREA_CNT int,OL_RFNDD_DEMO_CNT int,OL_RFNDD_OFFER_CNT int,OL_RTRNING_CUST_CNT int,OL_RTRNING_AREA_CNT int,OL_RTRNING_DEMO_CNT int,OL_RTRNING_OFFER_CNT int,OL_RTRWEB_PAGE_CNT int,OL_REASON_CNT int,OL_NET_LOSS int,OL_NET_LOSS_DM_A int,OL_NET_LOSS_DM_B int,OL_NET_LOSS_DM_C int)  STORED BY 'org.apache.carbondata.format' """).collect
 
   }
 
@@ -48,7 +48,7 @@
   test("OSCON_Query_CreateTableVIP", Include) {
     sql(s"""drop table if exists oscon_carbon_vip""").collect
 
-    sql(s"""CREATE TABLE oscon_carbon_vip (CUST_ID String,CUST_COUNTRY String,CUST_STATE String,CUST_CITY String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,WH_COUNTRY String,WH_STATE String,WH_CITY String,WH_COUNTY String,CUST_JOB_TITLE String,CUST_BUY_POTENTIAL String,PROD_UNQ_MDL_ID String,PROD_BRAND_NAME String,PRODUCT_NAME String,PRODUCT_MODEL String,PROD_MODEL_ID String,PROD_STD_CST int,REG_UNIT_PRICE int,TOTAL_PRD_COST int,TOTAL_PRD_DISC int,PROD_OFF_PRICE int,TOTAL_TX_AMT int,PROD_UNITS int,PROD_WEIGHT int,UNIT_PRICE_DSCNT_PCT int,DSCNT_AMT int,OL_SALES_PRICE int,PROD_COLOR String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,TRACKING_NO String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_NO String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_NICK_NAME String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_PRFRD_FLG String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_LOGIN String,CUST_EMAIL_ADDR String,CUST_LAST_RVW_DATE String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_BAR_CODE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_ZIP String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,CUST_DEP_COUNT int,CUST_VEHICLE_COUNT int,CUST_ADDRESS_CNT int,CUST_CRNT_CDEMO_CNT int,CUST_CRNT_HDEMO_CNT int,CUST_CRNT_ADDR_DM int,CUST_FIRST_SHIPTO_CNT int,CUST_FIRST_SALES_CNT int,CUST_GMT_OFFSET int,CUST_DEMO_CNT int,CUST_INCOME int,PROD_UNLIMITED int,EXTENDED_AMT int,FREIGHT_CHRG int,WAITING_PERIOD int,DELIVERY_PERIOD int,ITM_CRNT_PRICE int,ITM_UNITS int,ITM_WSLE_CST int,ITM_SIZE int,PRM_CST int,PRM_RESPONSE_TARGET int,PRM_ITM_DM int,SHP_MODE_CNT int,WH_GMT_OFFSET int,WH_SQ_FT int,STR_ORD_QTY int,STR_WSLE_CST int,STR_LIST_PRICE int,STR_SALES_PRICE int,STR_EXT_DSCNT_AMT int,STR_EXT_SALES_PRICE int,STR_EXT_WSLE_CST int,STR_EXT_LIST_PRICE int,STR_EXT_TX int,STR_COUPON_AMT int,STR_NET_PAID int,STR_NET_PAID_INC_TX int,STR_NET_PRFT int,STR_SOLD_YR_CNT int,STR_SOLD_MM_CNT int,STR_SOLD_ITM_CNT int,STR_TOTAL_CUST_CNT int,STR_AREA_CNT int,STR_DEMO_CNT int,STR_OFFER_CNT int,STR_PRM_CNT int,STR_TICKET_CNT int,STR_NET_PRFT_DM_A int,STR_NET_PRFT_DM_B int,STR_NET_PRFT_DM_C int,STR_NET_PRFT_DM_D int,STR_NET_PRFT_DM_E int,STR_RET_STR_ID int,STR_RET_REASON_CNT int,STR_RET_TICKET_NO int,STR_RTRN_QTY int,STR_RTRN_AMT int,STR_RTRN_TX int,STR_RTRN_AMT_INC_TX int,STR_RET_FEE int,STR_RTRN_SHIP_CST int,STR_RFNDD_CSH int,STR_REVERSED_CHRG int,STR_STR_CREDIT int,STR_RET_NET_LOSS int,STR_RTRNED_YR_CNT int,STR_RTRN_MM_CNT int,STR_RET_ITM_CNT int,STR_RET_CUST_CNT int,STR_RET_AREA_CNT int,STR_RET_OFFER_CNT int,STR_RET_PRM_CNT int,STR_RET_NET_LOSS_DM_A int,STR_RET_NET_LOSS_DM_B int,STR_RET_NET_LOSS_DM_C int,STR_RET_NET_LOSS_DM_D int,OL_ORD_QTY int,OL_WSLE_CST int,OL_LIST_PRICE int,OL_EXT_DSCNT_AMT int,OL_EXT_SALES_PRICE int,OL_EXT_WSLE_CST int,OL_EXT_LIST_PRICE int,OL_EXT_TX int,OL_COUPON_AMT int,OL_EXT_SHIP_CST int,OL_NET_PAID int,OL_NET_PAID_INC_TX int,OL_NET_PAID_INC_SHIP int,OL_NET_PAID_INC_SHIP_TX int,OL_NET_PRFT int,OL_SOLD_YR_CNT int,OL_SOLD_MM_CNT int,OL_SHIP_DATE_CNT int,OL_ITM_CNT int,OL_BILL_CUST_CNT int,OL_BILL_AREA_CNT int,OL_BILL_DEMO_CNT int,OL_BILL_OFFER_CNT int,OL_SHIP_CUST_CNT int,OL_SHIP_AREA_CNT int,OL_SHIP_DEMO_CNT int,OL_SHIP_OFFER_CNT int,OL_WEB_PAGE_CNT int,OL_WEB_SITE_CNT int,OL_SHIP_MODE_CNT int,OL_WH_CNT int,OL_PRM_CNT int,OL_NET_PRFT_DM_A int,OL_NET_PRFT_DM_B int,OL_NET_PRFT_DM_C int,OL_NET_PRFT_DM_D int,OL_RET_RTRN_QTY int,OL_RTRN_AMT int,OL_RTRN_TX int,OL_RTRN_AMT_INC_TX int,OL_RET_FEE int,OL_RTRN_SHIP_CST int,OL_RFNDD_CSH int,OL_REVERSED_CHRG int,OL_ACCOUNT_CREDIT int,OL_RTRNED_YR_CNT int,OL_RTRNED_MM_CNT int,OL_RTRITM_CNT int,OL_RFNDD_CUST_CNT int,OL_RFNDD_AREA_CNT int,OL_RFNDD_DEMO_CNT int,OL_RFNDD_OFFER_CNT int,OL_RTRNING_CUST_CNT int,OL_RTRNING_AREA_CNT int,OL_RTRNING_DEMO_CNT int,OL_RTRNING_OFFER_CNT int,OL_RTRWEB_PAGE_CNT int,OL_REASON_CNT int,OL_NET_LOSS int,OL_NET_LOSS_DM_A int,OL_NET_LOSS_DM_B int,OL_NET_LOSS_DM_C int)  STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='PROD_UNQ_MDL_ID,CUST_NICK_NAME,CUST_LOGIN,CUST_EMAIL_ADDR,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,PROD_BAR_CODE,TRACKING_NO,STR_ORDER_NO')""").collect
+    sql(s"""CREATE TABLE oscon_carbon_vip (CUST_ID String,CUST_COUNTRY String,CUST_STATE String,CUST_CITY String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,WH_COUNTRY String,WH_STATE String,WH_CITY String,WH_COUNTY String,CUST_JOB_TITLE String,CUST_BUY_POTENTIAL String,PROD_UNQ_MDL_ID String,PROD_BRAND_NAME String,PRODUCT_NAME String,PRODUCT_MODEL String,PROD_MODEL_ID String,PROD_STD_CST int,REG_UNIT_PRICE int,TOTAL_PRD_COST int,TOTAL_PRD_DISC int,PROD_OFF_PRICE int,TOTAL_TX_AMT int,PROD_UNITS int,PROD_WEIGHT int,UNIT_PRICE_DSCNT_PCT int,DSCNT_AMT int,OL_SALES_PRICE int,PROD_COLOR String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,TRACKING_NO String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_NO String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_NICK_NAME String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_PRFRD_FLG String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_LOGIN String,CUST_EMAIL_ADDR String,CUST_LAST_RVW_DATE String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_BAR_CODE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_ZIP String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,CUST_DEP_COUNT int,CUST_VEHICLE_COUNT int,CUST_ADDRESS_CNT int,CUST_CRNT_CDEMO_CNT int,CUST_CRNT_HDEMO_CNT int,CUST_CRNT_ADDR_DM int,CUST_FIRST_SHIPTO_CNT int,CUST_FIRST_SALES_CNT int,CUST_GMT_OFFSET int,CUST_DEMO_CNT int,CUST_INCOME int,PROD_UNLIMITED int,EXTENDED_AMT int,FREIGHT_CHRG int,WAITING_PERIOD int,DELIVERY_PERIOD int,ITM_CRNT_PRICE int,ITM_UNITS int,ITM_WSLE_CST int,ITM_SIZE int,PRM_CST int,PRM_RESPONSE_TARGET int,PRM_ITM_DM int,SHP_MODE_CNT int,WH_GMT_OFFSET int,WH_SQ_FT int,STR_ORD_QTY int,STR_WSLE_CST int,STR_LIST_PRICE int,STR_SALES_PRICE int,STR_EXT_DSCNT_AMT int,STR_EXT_SALES_PRICE int,STR_EXT_WSLE_CST int,STR_EXT_LIST_PRICE int,STR_EXT_TX int,STR_COUPON_AMT int,STR_NET_PAID int,STR_NET_PAID_INC_TX int,STR_NET_PRFT int,STR_SOLD_YR_CNT int,STR_SOLD_MM_CNT int,STR_SOLD_ITM_CNT int,STR_TOTAL_CUST_CNT int,STR_AREA_CNT int,STR_DEMO_CNT int,STR_OFFER_CNT int,STR_PRM_CNT int,STR_TICKET_CNT int,STR_NET_PRFT_DM_A int,STR_NET_PRFT_DM_B int,STR_NET_PRFT_DM_C int,STR_NET_PRFT_DM_D int,STR_NET_PRFT_DM_E int,STR_RET_STR_ID int,STR_RET_REASON_CNT int,STR_RET_TICKET_NO int,STR_RTRN_QTY int,STR_RTRN_AMT int,STR_RTRN_TX int,STR_RTRN_AMT_INC_TX int,STR_RET_FEE int,STR_RTRN_SHIP_CST int,STR_RFNDD_CSH int,STR_REVERSED_CHRG int,STR_STR_CREDIT int,STR_RET_NET_LOSS int,STR_RTRNED_YR_CNT int,STR_RTRN_MM_CNT int,STR_RET_ITM_CNT int,STR_RET_CUST_CNT int,STR_RET_AREA_CNT int,STR_RET_OFFER_CNT int,STR_RET_PRM_CNT int,STR_RET_NET_LOSS_DM_A int,STR_RET_NET_LOSS_DM_B int,STR_RET_NET_LOSS_DM_C int,STR_RET_NET_LOSS_DM_D int,OL_ORD_QTY int,OL_WSLE_CST int,OL_LIST_PRICE int,OL_EXT_DSCNT_AMT int,OL_EXT_SALES_PRICE int,OL_EXT_WSLE_CST int,OL_EXT_LIST_PRICE int,OL_EXT_TX int,OL_COUPON_AMT int,OL_EXT_SHIP_CST int,OL_NET_PAID int,OL_NET_PAID_INC_TX int,OL_NET_PAID_INC_SHIP int,OL_NET_PAID_INC_SHIP_TX int,OL_NET_PRFT int,OL_SOLD_YR_CNT int,OL_SOLD_MM_CNT int,OL_SHIP_DATE_CNT int,OL_ITM_CNT int,OL_BILL_CUST_CNT int,OL_BILL_AREA_CNT int,OL_BILL_DEMO_CNT int,OL_BILL_OFFER_CNT int,OL_SHIP_CUST_CNT int,OL_SHIP_AREA_CNT int,OL_SHIP_DEMO_CNT int,OL_SHIP_OFFER_CNT int,OL_WEB_PAGE_CNT int,OL_WEB_SITE_CNT int,OL_SHIP_MODE_CNT int,OL_WH_CNT int,OL_PRM_CNT int,OL_NET_PRFT_DM_A int,OL_NET_PRFT_DM_B int,OL_NET_PRFT_DM_C int,OL_NET_PRFT_DM_D int,OL_RET_RTRN_QTY int,OL_RTRN_AMT int,OL_RTRN_TX int,OL_RTRN_AMT_INC_TX int,OL_RET_FEE int,OL_RTRN_SHIP_CST int,OL_RFNDD_CSH int,OL_REVERSED_CHRG int,OL_ACCOUNT_CREDIT int,OL_RTRNED_YR_CNT int,OL_RTRNED_MM_CNT int,OL_RTRITM_CNT int,OL_RFNDD_CUST_CNT int,OL_RFNDD_AREA_CNT int,OL_RFNDD_DEMO_CNT int,OL_RFNDD_OFFER_CNT int,OL_RTRNING_CUST_CNT int,OL_RTRNING_AREA_CNT int,OL_RTRNING_DEMO_CNT int,OL_RTRNING_OFFER_CNT int,OL_RTRWEB_PAGE_CNT int,OL_REASON_CNT int,OL_NET_LOSS int,OL_NET_LOSS_DM_A int,OL_NET_LOSS_DM_B int,OL_NET_LOSS_DM_C int)  STORED BY 'org.apache.carbondata.format' """).collect
 
   }
 
@@ -285,7 +285,7 @@
     sql(s"""drop table if exists cmb""").collect
     sql(s"""drop table if exists cmb_hive""").collect
 
-    sql(s"""CREATE table cmb (Cust_UID String,year String, month String, companyNumber String, familyadNumber String,  companyAddress String, company String, occupation String, certicardValidTime String, race String, CerticardCity String, birthday String, VIPLevel String, ageRange String, familyaddress String, dimension16 String, SubsidaryBank String, AccountCreationTime String, dimension19 String, dimension20 String, DemandDeposits double, TimeDeposits double, financial double, TreasuryBonds double, fund double, incomeOneyear double, outcomeOneyear double, insurance double, Goldaccount double, dollarDeposits int, euroDeposits int, euroDeposits1 double, euroDeposits2 double, yenDeposits int, wonDeposits int, rupeeDeposits double, HongKongDeposits double, numberoftransactions int, measure19 double, measure20 double, measure21 int, measure22 double, measure23 double, measure24 int, measure25 double, measure26 double, measure27 int, measure28 double, measure29 int, measure30 double, measure31 double, measure32 double, measure33 double, measure34 int, measure35 double, measure36 double, measure37 int, measure38 double, measure39 double, measure40 int, measure41 double, measure42 double, measure43 int, measure44 double, measure45 int, measure46 double, measure47 int, measure48 double, measure49 int, measure50 double, measure51 int, measure52 double, measure53 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_INCLUDE'='dollarDeposits,euroDeposits,yenDeposits,wonDeposits,numberoftransactions,measure21,measure24,measure27,measure29,measure34,measure37,measure40,measure43,measure45,measure47,measure49,measure51,measure53')""").collect
+    sql(s"""CREATE table cmb (Cust_UID String,year String, month String, companyNumber String, familyadNumber String,  companyAddress String, company String, occupation String, certicardValidTime String, race String, CerticardCity String, birthday String, VIPLevel String, ageRange String, familyaddress String, dimension16 String, SubsidaryBank String, AccountCreationTime String, dimension19 String, dimension20 String, DemandDeposits double, TimeDeposits double, financial double, TreasuryBonds double, fund double, incomeOneyear double, outcomeOneyear double, insurance double, Goldaccount double, dollarDeposits int, euroDeposits int, euroDeposits1 double, euroDeposits2 double, yenDeposits int, wonDeposits int, rupeeDeposits double, HongKongDeposits double, numberoftransactions int, measure19 double, measure20 double, measure21 int, measure22 double, measure23 double, measure24 int, measure25 double, measure26 double, measure27 int, measure28 double, measure29 int, measure30 double, measure31 double, measure32 double, measure33 double, measure34 int, measure35 double, measure36 double, measure37 int, measure38 double, measure39 double, measure40 int, measure41 double, measure42 double, measure43 int, measure44 double, measure45 int, measure46 double, measure47 int, measure48 double, measure49 int, measure50 double, measure51 int, measure52 double, measure53 int) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""CREATE table cmb_hive (Cust_UID String,year String, month String, companyNumber String, familyadNumber String,  companyAddress String, company String, occupation String, certicardValidTime String, race String, CerticardCity String, birthday String, VIPLevel String, ageRange String, familyaddress String, dimension16 String, SubsidaryBank String, AccountCreationTime String, dimension19 String, dimension20 String, DemandDeposits double, TimeDeposits double, financial double, TreasuryBonds double, fund double, incomeOneyear double, outcomeOneyear double, insurance double, Goldaccount double, dollarDeposits int, euroDeposits int, euroDeposits1 double, euroDeposits2 double, yenDeposits int, wonDeposits int, rupeeDeposits double, HongKongDeposits double, numberoftransactions int, measure19 double, measure20 double, measure21 int, measure22 double, measure23 double, measure24 int, measure25 double, measure26 double, measure27 int, measure28 double, measure29 int, measure30 double, measure31 double, measure32 double, measure33 double, measure34 int, measure35 double, measure36 double, measure37 int, measure38 double, measure39 double, measure40 int, measure41 double, measure42 double, measure43 int, measure44 double, measure45 int, measure46 double, measure47 int, measure48 double, measure49 int, measure50 double, measure51 int, measure52 double, measure53 int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesRangeFilterTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesRangeFilterTestCase.scala
index bac4697..8480eb5 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesRangeFilterTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesRangeFilterTestCase.scala
@@ -43,7 +43,7 @@
   sql("drop table if exists NO_DICTIONARY_CARBON")
   sql("drop table if exists NO_DICTIONARY_CARBON_hive")
 
-  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON(empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno, empname,designation')""").collect
+  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON(empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' """).collect
 
   sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_hive(empno string, empname String, designation String,doj Timestamp, workgroupcategory Int,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp,attendance Int,utilization Int,salary Int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
@@ -51,7 +51,7 @@
 
   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/RangeFilter/rangefilterdata_hive1.csv' INTO TABLE NO_DICTIONARY_CARBON_hive """).collect
 
-  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_6 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno, empname,designation')""").collect
+  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_6 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' """).collect
 
   sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_6_hive (empno string, empname String, designation String,doj Timestamp, workgroupcategory Int,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp,attendance Int,utilization Int,salary Int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
@@ -59,7 +59,7 @@
 
   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/RangeFilter/rangefilterdata_hive2.csv' INTO TABLE NO_DICTIONARY_CARBON_6_hive """).collect
 
-  sql(s"""CREATE TABLE DICTIONARY_CARBON_6 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate timestamp,projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='empname,designation')""").collect
+  sql(s"""CREATE TABLE DICTIONARY_CARBON_6 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate timestamp,projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' """).collect
 
   sql(s"""CREATE TABLE DICTIONARY_CARBON_6_hive (empno string, empname String, designation String,doj Timestamp, workgroupcategory Int,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp,attendance Int,utilization Int,salary Int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
@@ -67,7 +67,7 @@
 
   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/RangeFilter/rangefilterdata_hive3.csv' INTO TABLE DICTIONARY_CARBON_6_hive """).collect
 
-  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_7 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')""").collect
+  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_7 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' """).collect
 
   sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_7_hive (empno string, empname String, designation String,doj Timestamp, workgroupcategory Int,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp,attendance Int,utilization Int,salary Int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
@@ -75,7 +75,7 @@
 
   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/RangeFilter/rangefilterdata_hive4.csv' INTO TABLE NO_DICTIONARY_CARBON_7_hive  """).collect
 
-  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_8 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')""").collect
+  sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_8 (empno string, doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp, designation String,attendance Int,utilization Int,salary Int) STORED BY 'org.apache.carbondata.format' """).collect
 
   sql(s"""CREATE TABLE NO_DICTIONARY_CARBON_8_hive (empno string, empname String, designation String,doj Timestamp, workgroupcategory Int,workgroupcategoryname String, deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, projectenddate Timestamp,attendance Int,utilization Int,salary Int)  ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesSparkBlockDistTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesSparkBlockDistTestCase.scala
index a5004eb..da7ffda 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesSparkBlockDistTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/QueriesSparkBlockDistTestCase.scala
@@ -32,7 +32,7 @@
   test("BlockDist_PTS001_TC002123", Include) {
     sql("drop table if exists flow_carbon_256b")
     sql("drop table if exists flow_carbon_256b_hive")
-    sql(s"""CREATE TABLE IF NOT EXISTS  flow_carbon_256b ( txn_dte     String, dt  String, txn_bk      String, txn_br      String, own_bk      String, own_br      String, opp_bk      String, bus_opr_cde String, opt_prd_cde String, cus_no      String, cus_ac      String, opp_ac_nme  String, opp_ac      String, bv_no       String, aco_ac      String, ac_dte      String, txn_cnt     int,     jrn_par     int,     mfm_jrn_no  String,     cbn_jrn_no  String,     ibs_jrn_no  String,     vch_no      String, vch_seq     String,     srv_cde     String, bus_cd_no   String, id_flg      String, bv_cde      String, txn_time    String, txn_tlr     String, ety_tlr     String, ety_bk      String, ety_br      String, bus_pss_no  String, chk_flg     String, chk_tlr     String, chk_jrn_no  String,     bus_sys_no  String, txn_sub_cde String, fin_bus_cde String, fin_bus_sub_cde     String, chl         String, tml_id      String, sus_no      String, sus_seq     String,     cho_seq     String,     itm_itm     String, itm_sub     String, itm_sss     String, dc_flg      String, amt         decimal(15,2), bal         decimal(15,2), ccy         String, spv_flg     String, vch_vld_dte String, pst_bk      String, pst_br      String, ec_flg      String, aco_tlr     String, gen_flg     String, his_rec_sum_flg     String, his_flg     String, vch_typ     String, val_dte     String, opp_ac_flg  String, cmb_flg     String, ass_vch_flg String, cus_pps_flg String, bus_rmk_cde String, vch_bus_rmk String, tec_rmk_cde String, vch_tec_rmk String, rsv_ara     String, gems_last_upd_d     String, gems_last_upd_d_bat String, maps_date   String, maps_job    String ) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='dt,jrn_par,txn_bk,txn_br,ety_bk,ety_br,chk_flg,bus_sys_no,bus_opr_cde,chl,sus_no,itm_itm,itm_sub,itm_sss,dc_flg,ccy,spv_flg,pst_bk,pst_br,ec_flg,gen_flg,his_rec_sum_flg,his_flg,vch_typ,opp_ac_flg,cmb_flg,ass_vch_flg,cus_pps_flg,bus_rmk_cde,vch_bus_rmk,tec_rmk_cde,vch_tec_rmk,rsv_ara,own_br,own_bk','DICTIONARY_EXCLUDE'='aco_ac,ac_dte,mfm_jrn_no,cbn_jrn_no,ibs_jrn_no,vch_no,vch_seq,srv_cde,cus_no,bus_cd_no,id_flg,cus_ac,bv_cde,bv_no,txn_dte,txn_time,txn_tlr,ety_tlr,bus_pss_no,chk_tlr,chk_jrn_no,txn_sub_cde,fin_bus_cde,fin_bus_sub_cde,opt_prd_cde,tml_id,sus_seq,cho_seq,vch_vld_dte,aco_tlr,opp_ac,opp_ac_nme,opp_bk,val_dte,gems_last_upd_d,gems_last_upd_d_bat,maps_date,maps_job')""").collect
+    sql(s"""CREATE TABLE IF NOT EXISTS  flow_carbon_256b ( txn_dte     String, dt  String, txn_bk      String, txn_br      String, own_bk      String, own_br      String, opp_bk      String, bus_opr_cde String, opt_prd_cde String, cus_no      String, cus_ac      String, opp_ac_nme  String, opp_ac      String, bv_no       String, aco_ac      String, ac_dte      String, txn_cnt     int,     jrn_par     int,     mfm_jrn_no  String,     cbn_jrn_no  String,     ibs_jrn_no  String,     vch_no      String, vch_seq     String,     srv_cde     String, bus_cd_no   String, id_flg      String, bv_cde      String, txn_time    String, txn_tlr     String, ety_tlr     String, ety_bk      String, ety_br      String, bus_pss_no  String, chk_flg     String, chk_tlr     String, chk_jrn_no  String,     bus_sys_no  String, txn_sub_cde String, fin_bus_cde String, fin_bus_sub_cde     String, chl         String, tml_id      String, sus_no      String, sus_seq     String,     cho_seq     String,     itm_itm     String, itm_sub     String, itm_sss     String, dc_flg      String, amt         decimal(15,2), bal         decimal(15,2), ccy         String, spv_flg     String, vch_vld_dte String, pst_bk      String, pst_br      String, ec_flg      String, aco_tlr     String, gen_flg     String, his_rec_sum_flg     String, his_flg     String, vch_typ     String, val_dte     String, opp_ac_flg  String, cmb_flg     String, ass_vch_flg String, cus_pps_flg String, bus_rmk_cde String, vch_bus_rmk String, tec_rmk_cde String, vch_tec_rmk String, rsv_ara     String, gems_last_upd_d     String, gems_last_upd_d_bat String, maps_date   String, maps_job    String ) STORED BY 'org.apache.carbondata.format' """).collect
 
     sql(s"""CREATE TABLE IF NOT EXISTS  flow_carbon_256b_hive ( txn_dte     String, dt  String, txn_bk      String, txn_br      String, own_bk      String, own_br      String, opp_bk      String, bus_opr_cde String, opt_prd_cde String, cus_no      String, cus_ac      String, opp_ac_nme  String, opp_ac      String, bv_no       String, aco_ac      String, ac_dte      String, txn_cnt     int,     jrn_par     int,     mfm_jrn_no  String,     cbn_jrn_no  String,     ibs_jrn_no  String,     vch_no      String, vch_seq     String,     srv_cde     String, bus_cd_no   String, id_flg      String, bv_cde      String, txn_time    String, txn_tlr     String, ety_tlr     String, ety_bk      String, ety_br      String, bus_pss_no  String, chk_flg     String, chk_tlr     String, chk_jrn_no  String,     bus_sys_no  String, txn_sub_cde String, fin_bus_cde String, fin_bus_sub_cde     String, chl         String, tml_id      String, sus_no      String, sus_seq     String,     cho_seq     String,     itm_itm     String, itm_sub     String, itm_sss     String, dc_flg      String, amt         decimal(15,2), bal         decimal(15,2), ccy         String, spv_flg     String, vch_vld_dte String, pst_bk      String, pst_br      String, ec_flg      String, aco_tlr     String, gen_flg     String, his_rec_sum_flg     String, his_flg     String, vch_typ     String, val_dte     String, opp_ac_flg  String, cmb_flg     String, ass_vch_flg String, cus_pps_flg String, bus_rmk_cde String, vch_bus_rmk String, tec_rmk_cde String, vch_tec_rmk String, rsv_ara     String, gems_last_upd_d     String, gems_last_upd_d_bat String, maps_date   String, maps_job    String )  ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\n'""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SetParameterTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SetParameterTestCase.scala
index 54d9e3f..d444fde 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SetParameterTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SetParameterTestCase.scala
@@ -37,7 +37,6 @@
     sql("drop table if exists carbon_table")
     sql("drop table if exists emptyColumnValues")
     sql("drop table if exists carbon_table_bad_record_logger")
-    sql("drop table if exists carbon_table_single_pass")
     sql("drop table if exists carbon_table_disable_bad_record_logger")
     sql("drop table if exists carbon_table_load")
     sqlContext.sparkSession.catalog.clearCache()
@@ -166,7 +165,7 @@
       """)
     sql(
       s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/badrecord/doubleqoute.csv' into table
-         |emptyColumnValues options('SINGLE_PASS'='true')"""
+         |emptyColumnValues """
         .stripMargin)
     checkAnswer(
       s"""select count(*) from emptyColumnValues""",
@@ -185,24 +184,13 @@
       """)
     sql(
       s"""LOAD DATA LOCAL INPATH '$resourcesPath/Data/badrecord/doubleqoute.csv' into table
-         |emptyColumnValues options('SINGLE_PASS'='true')"""
+         |emptyColumnValues """
         .stripMargin)
     checkAnswer(
       s"""select count(*) from emptyColumnValues""",
       Seq(Row(1)), "SetParameterTestCase-TC_008-test SET property IS__EMPTY_DATA_BAD_RECORD=TRUE")
   }
 
-  test("TC_009-test SET property for Single Pass") {
-    sql("drop table if exists carbon_table_single_pass")
-    sql("SET carbon.options.single.pass=true")
-    sql(
-      "create table carbon_table_single_pass(empno int, empname String, designation String, doj " +
-      "Timestamp,workgroupcategory int, workgroupcategoryname String, deptno int, deptname " +
-      "String," +
-      "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
-      "utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
-  }
-
   test("TC_010-test SET property for Sort Scope-Local_Sort") {
     sql("drop table if exists carbon_table")
     sql("SET carbon.options.bad.records.logger.enable=true")
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SinglepassTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SinglepassTestCase.scala
deleted file mode 100644
index 6487696..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SinglepassTestCase.scala
+++ /dev/null
@@ -1,765 +0,0 @@
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.cluster.sdv.generated
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util._
-import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.TestQueryExecutor
-
-/**
- * Test Class for singlepassTestCase to verify all scenerios
- */
-
-class SinglepassTestCase extends QueryTest with BeforeAndAfterAll {
-         
-
-  //To check data loading with OPTIONS ‘SINGLE_PASS’=’true’
-  test("Loading-004-01-01-01_001-TC_001", Include) {
-     sql(s"""drop table if exists test1""").collect
-   sql(s"""create table test1(imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId double,productionDate Timestamp,deliveryDate timestamp,deliverycharge double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('table_blocksize'='1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test1 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test1""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_001")
-     sql(s"""drop table test1""").collect
-  }
-
-
-  //To check data loading with OPTIONS ‘SINGLE_PASS’=’false’
-  test("Loading-004-01-01-01_001-TC_002", Include) {
-     sql(s"""create table test1(imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId double,productionDate Timestamp,deliveryDate timestamp,deliverycharge double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('table_blocksize'='1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test1 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='FALSE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test1""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_002")
-
-  }
-
-
-  //To check data loading from CSV with incomplete data
-  test("Loading-004-01-01-01_001-TC_003", Include) {
-    intercept[Exception] {
-     sql(s"""drop table if exists uniqdata""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-      sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_incomplete.csv' INTO TABLE uniqdata OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    }
-  }
-
-
-  //To check data loading from CSV with bad records
-  test("Loading-004-01-01-01_001-TC_004", Include) {
-    intercept[Exception] {
-      sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_badrec.csv' INTO TABLE uniqdata OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    }
-  }
-
-
-  //To check data loading from CSV with no data
-  test("Loading-004-01-01-01_001-TC_005", Include) {
-    intercept[Exception] {
-      sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_nodata.csv' INTO TABLE uniqdata OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    }
-  }
-
-
-  //To check data loading from CSV with incomplete data
-  test("Loading-004-01-01-01_001-TC_006", Include) {
-    intercept[Exception] {
-      sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_incomplete.csv' INTO TABLE uniqdata OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='FALSE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    }
-  }
-
-
-  //To check data loading from CSV with wrong data
-  test("Loading-004-01-01-01_001-TC_007", Include) {
-    intercept[Exception] {
-      sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_incomplete.csv' INTO TABLE uniqdata OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='FALSE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    }
-  }
-
-
-  //To check data loading from CSV with no data and 'SINGLEPASS' = 'FALSE'
-  test("Loading-004-01-01-01_001-TC_008", Include) {
-    intercept[Exception] {
-      sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_nodata.csv.csv' INTO TABLE uniqdata OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='FALSE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    }
-  }
-
-
-  //To check data loading using  'SINGLE_PASS'='NULL/any invalid string'
-  test("Loading-004-01-01-01_001-TC_009", Include) {
-     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test1 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='NULL', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test1""",
-      Seq(Row(198)), "singlepassTestCase_Loading-004-01-01-01_001-TC_009")
-     sql(s"""drop table test1""").collect
-  }
-
-
-  //To check data load using multiple CSV from folder into table with single_pass=true
-  test("Loading-004-01-01-01_001-TC_010", Include) {
-     sql(s"""drop table if exists emp_record12""").collect
-   sql(s"""create table emp_record12 (ID int,Name string,DOJ timestamp,Designation string,Salary double,Dept string,DOB timestamp,Addr string,Gender string,Mob bigint) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA inpath '$resourcesPath/Data/singlepass/data' into table emp_record12 options('DELIMITER'=',', 'QUOTECHAR'='"','SINGLE_PASS'='TRUE','FILEHEADER'='ID,Name,DOJ,Designation,Salary,Dept,DOB,Addr,Gender,Mob','BAD_RECORDS_ACTION'='FORCE')""").collect
-    sql(s"""select count(*) from emp_record12""").collect
-
-     sql(s"""drop table emp_record12""").collect
-  }
-
-
-  //To check data load using CSV from multiple level of folders into table
-  test("Loading-004-01-01-01_001-TC_011", Include) {
-     sql(s"""create table emp_record12 (ID int,Name string,DOJ timestamp,Designation string,Salary double,Dept string,DOB timestamp,Addr string,Gender string,Mob bigint) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA inpath '$resourcesPath/Data/singlepass/data' into table emp_record12 options('DELIMITER'=',', 'QUOTECHAR'='"','SINGLE_PASS'='TRUE','FILEHEADER'='ID,Name,DOJ,Designation,Salary,Dept,DOB,Addr,Gender,Mob','BAD_RECORDS_ACTION'='FORCE')""").collect
-    sql(s"""select count(*) from emp_record12""").collect
-
-     sql(s"""drop table emp_record12""").collect
-  }
-
-
-  //To check data load using multiple CSV from folder into table with single_pass=false
-  test("Loading-004-01-01-01_001-TC_012", Include) {
-     sql(s"""create table emp_record12 (ID int,Name string,DOJ timestamp,Designation string,Salary double,Dept string,DOB timestamp,Addr string,Gender string,Mob bigint) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA inpath '$resourcesPath/Data/singlepass/data' into table emp_record12 options('DELIMITER'=',', 'QUOTECHAR'='"','SINGLE_PASS'='FALSE','FILEHEADER'='ID,Name,DOJ,Designation,Salary,Dept,DOB,Addr,Gender,Mob','BAD_RECORDS_ACTION'='FORCE')""").collect
-    sql(s"""select count(*) from emp_record12""").collect
-
-     sql(s"""drop table emp_record12""").collect
-  }
-
-
-  //To check data load using CSV from multiple level of folders into table
-  test("Loading-004-01-01-01_001-TC_013", Include) {
-     sql(s"""create table emp_record12 (ID int,Name string,DOJ timestamp,Designation string,Salary double,Dept string,DOB timestamp,Addr string,Gender string,Mob bigint) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA inpath '$resourcesPath/Data/singlepass/data' into table emp_record12 options('DELIMITER'=',', 'QUOTECHAR'='"','SINGLE_PASS'='FALSE','FILEHEADER'='ID,Name,DOJ,Designation,Salary,Dept,DOB,Addr,Gender,Mob','BAD_RECORDS_ACTION'='FORCE')""").collect
-    sql(s"""select count(*) from emp_record12""").collect
-
-     sql(s"""drop table emp_record12""").collect
-  }
-
-
-  //To check Data loading in proper CSV format with .dat
-  test("Loading-004-01-01-01_001-TC_014", Include) {
-     sql(s"""drop table if exists uniqdata_file_extn""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.dat' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_014")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading in proper CSV format with .xls
-  test("Loading-004-01-01-01_001-TC_015", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.xls' into table uniqdata_file_extn OPTIONS('DELIMITER'='\001' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_015")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading in proper CSV format  with .doc
-  test("Loading-004-01-01-01_001-TC_016", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.dat' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_016")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading in proper CSV format  with .txt
-  test("Loading-004-01-01-01_001-TC_017", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.txt' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_017")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-
-  //To check Data loading in proper CSV format  wiithout any extension
-  test("Loading-004-01-01-01_001-TC_020", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_020")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading in proper CSV format with .dat with single_pass=false
-  test("Loading-004-01-01-01_001-TC_021", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.dat' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_021")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading in proper CSV format with .xls with single_pass=false
-  test("Loading-004-01-01-01_001-TC_022", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.xls' into table uniqdata_file_extn OPTIONS('DELIMITER'='\001' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_022")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-
-  //To check Data loading in proper CSV format  with .txt with single_pass=false
-  test("Loading-004-01-01-01_001-TC_024", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData.txt' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_024")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading in proper CSV format  wiithout any extension with single_pass=false
-  test("Loading-004-01-01-01_001-TC_027", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_file_extn (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-   sql(s"""LOAD DATA  inpath '$resourcesPath/Data/singlepass/2000_UniqData' into table uniqdata_file_extn OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_file_extn""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_027")
-     sql(s"""drop table uniqdata_file_extn""").collect
-  }
-
-
-  //To check Data loading with delimiters  as / [slash]
-  test("Loading-004-01-01-01_001-TC_028", Include) {
-     sql(s"""drop table if exists uniqdata_slash""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_slash(CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_slash.csv' into table uniqdata_slash OPTIONS('DELIMITER'='/' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_slash""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_028")
-     sql(s"""drop table uniqdata_slash""").collect
-  }
-
-
-  //To check Data loading with delimiters  as " [double quote]
-  test("Loading-004-01-01-01_001-TC_029", Include) {
-     sql(s"""drop table if exists uniqdata_doublequote""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_doublequote (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_quote.csv' into table uniqdata_doublequote OPTIONS('DELIMITER'='"' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_doublequote""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_029")
-     sql(s"""drop table uniqdata_doublequote""").collect
-  }
-
-
-  //To check Data loading with delimiters  as  ! [exclamation]
-  test("Loading-004-01-01-01_001-TC_030", Include) {
-     sql(s"""drop table if exists uniqdata_exclamation""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_exclamation (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_exclamation.csv' into table uniqdata_exclamation OPTIONS('DELIMITER'='!' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_exclamation""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_030")
-     sql(s"""drop table uniqdata_exclamation""").collect
-  }
-
-
-  //To check Data loading with delimiters  as  | [pipeline]
-  test("Loading-004-01-01-01_001-TC_031", Include) {
-     sql(s"""drop table if exists uniqdata_pipe""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_pipe (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_pipe.csv' into table uniqdata_pipe OPTIONS('DELIMITER'='|' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_pipe""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_031")
-     sql(s"""drop table uniqdata_pipe""").collect
-  }
-
-
-  //To check Data loading with delimiters  as ' [single quota]
-  test("Loading-004-01-01-01_001-TC_032", Include) {
-     sql(s"""drop table if exists uniqdata_singleQuote""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_singleQuote (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_singlequote.csv' into table uniqdata_singleQuote OPTIONS('DELIMITER'="'" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_singleQuote""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_032")
-     sql(s"""drop table uniqdata_singleQuote""").collect
-  }
-
-
-  //To check Data loading with delimiters  as \017
-  test("Loading-004-01-01-01_001-TC_033", Include) {
-     sql(s"""drop table if exists uniqdata_017""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_017 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_017.csv' into table uniqdata_017 OPTIONS('DELIMITER'="\017" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_017""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_033")
-     sql(s"""drop table uniqdata_017""").collect
-  }
-
-
-  //To check Data loading with delimiters  as \001
-  test("Loading-004-01-01-01_001-TC_034", Include) {
-     sql(s"""drop table if exists uniqdata_001""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_001 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_001.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_001""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_034")
-     sql(s"""drop table uniqdata_001""").collect
-  }
-
-
-  //To check Data loading with delimiters  as / [slash]  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_035", Include) {
-     sql(s"""drop table if exists uniqdata_slash""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_slash(CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_slash.csv' into table uniqdata_slash OPTIONS('DELIMITER'='/' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_slash""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_035")
-     sql(s"""drop table uniqdata_slash""").collect
-  }
-
-
-  //To check Data loading with delimiters  as " [double quote]  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_036", Include) {
-     sql(s"""drop table if exists uniqdata_doublequote""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_doublequote (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_quote.csv' into table uniqdata_doublequote OPTIONS('DELIMITER'='"' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_doublequote""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_036")
-     sql(s"""drop table uniqdata_doublequote""").collect
-  }
-
-
-  //To check Data loading with delimiters  as  ! [exclamation]  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_037", Include) {
-     sql(s"""drop table if exists uniqdata_exclamation""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_exclamation (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_exclamation.csv' into table uniqdata_exclamation OPTIONS('DELIMITER'='!' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_exclamation""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_037")
-     sql(s"""drop table uniqdata_exclamation""").collect
-  }
-
-
-  //To check Data loading with delimiters  as  | [pipeline]  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_038", Include) {
-     sql(s"""drop table if exists uniqdata_pipe""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_pipe (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_pipe.csv' into table uniqdata_pipe OPTIONS('DELIMITER'='|' , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_pipe""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_038")
-     sql(s"""drop table uniqdata_pipe""").collect
-  }
-
-
-  //To check Data loading with delimiters  as ' [single quota]  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_039", Include) {
-     sql(s"""drop table if exists uniqdata_singleQuote""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_singleQuote (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_singlequote.csv' into table uniqdata_singleQuote OPTIONS('DELIMITER'="'" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_singleQuote""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_039")
-     sql(s"""drop table uniqdata_singleQuote""").collect
-  }
-
-
-  //To check Data loading with delimiters  as \017  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_040", Include) {
-     sql(s"""drop table if exists uniqdata_017""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_017 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_017.csv' into table uniqdata_017 OPTIONS('DELIMITER'="\017" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_017""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_040")
-     sql(s"""drop table uniqdata_017""").collect
-  }
-
-
-  //To check Data loading with delimiters  as \001  and SINGLE_PASS= FALSE
-  test("Loading-004-01-01-01_001-TC_041", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_001 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData_001.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-    checkAnswer(s"""select count(*) from uniqdata_001""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_041")
-     sql(s"""drop table uniqdata_001""").collect
-  }
-
-
-  //To check Auto compaction is successful with carbon.enable.auto.load.merge= True & SINGLE_PASS=TRUE
-  test("Loading-004-01-01-01_001-TC_043", Include) {
-     sql(s"""drop table if exists uniqdata_001""").collect
-   sql(s"""CREATE TABLE if not exists uniqdata_001 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-   sql(s"""alter table uniqdata_001 compact 'minor'""").collect
-    sql(s"""show segments for table uniqdata_001""").collect
-     sql(s"""drop table uniqdata_001""").collect
-  }
-
-
-  //To check Auto compaction is successful with carbon.enable.auto.load.merge= True & SINGLE_PASS=FALSE
-  test("Loading-004-01-01-01_001-TC_044", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_001 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-   sql(s"""alter table uniqdata_001 compact 'minor'""").collect
-    sql(s"""show segments for table uniqdata_001""").collect
-     sql(s"""drop table uniqdata_001""").collect
-  }
-
-
-  //To check Auto compaction is successful with carbon.enable.auto.load.merge= false & SINGLE_PASS=TRUE
-  test("Loading-004-01-01-01_001-TC_045", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_001 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-
-   sql(s"""alter table uniqdata_001 compact 'major'""").collect
-    sql(s"""show segments for table uniqdata_001""").collect
-     sql(s"""drop table uniqdata_001""").collect
-  }
-
-
-  //To check Auto compaction is successful with carbon.enable.auto.load.merge= false & SINGLE_PASS=FALSE
-  test("Loading-004-01-01-01_001-TC_046", Include) {
-     sql(s"""CREATE TABLE if not exists uniqdata_001 (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/2000_UniqData.csv' into table uniqdata_001 OPTIONS('DELIMITER'="\001" , 'QUOTECHAR'='"', 'BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','COMPLEX_DELIMITER_LEVEL_1'='#', 'COMPLEX_DELIMITER_LEVEL_2'=':')""").collect
-
-   sql(s"""alter table uniqdata_001 compact 'major'""").collect
-    sql(s"""show segments for table uniqdata_001""").collect
-     sql(s"""drop table uniqdata_001""").collect
-  }
-
-
-  //To check Data loading is success with 'SINGLE_PASS'='TRUE' with already created table with Include dictionary
-  test("Loading-004-01-01-01_001-TC_051", Include) {
-     sql(s"""create database includeexclude""").collect
-   sql(s"""use includeexclude""").collect
-   sql(s"""create table test2 (imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId int,productionDate Timestamp,deliveryDate timestamp,deliverycharge decimal(10,2)) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='gamePointId,deviceInformationId')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test2 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test2""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_051")
-     sql(s"""drop table includeexclude.test2""").collect
-   sql(s"""drop database includeexclude cascade""").collect
-  }
-
-
-  //To check Data loading is success with 'SINGLE_PASS'='FALSE' with already created table with Include dictionary
-  test("Loading-004-01-01-01_001-TC_052", Include) {
-     sql(s"""create database includeexclude""").collect
-   sql(s"""use includeexclude""").collect
-   sql(s"""create table test2 (imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId int,productionDate Timestamp,deliveryDate timestamp,deliverycharge decimal(10,2)) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='gamePointId,deviceInformationId')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test2 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='FALSE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test2""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_052")
-     sql(s"""drop table includeexclude.test2""").collect
-   sql(s"""use default""").collect
-   sql(s"""drop database includeexclude cascade""").collect
-  }
-
-
-  //To check Data loading is success with 'SINGLE_PASS'='TRUE' with already created table with Exclude dictionary
-  test("Loading-004-01-01-01_001-TC_053", Include) {
-     sql(s"""drop table if exists test2""").collect
-   sql(s"""create table test2 (imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId int,productionDate Timestamp,deliveryDate timestamp,deliverycharge decimal(10,2)) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='imei,channelsId,AMSize,ActiveCountry,Activecity')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test2 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test2""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_053")
-     sql(s"""drop table test2""").collect
-  }
-
-
-  //To check Data loading is success with 'SINGLE_PASS'='FALSE' with already created table with Exclude dictionary
-  test("Loading-004-01-01-01_001-TC_054", Include) {
-     sql(s"""create table test2 (imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId int,productionDate Timestamp,deliveryDate timestamp,deliverycharge decimal(10,2)) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='imei,channelsId,AMSize,ActiveCountry,Activecity')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test2 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='FALSE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test2""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_054")
-     sql(s"""drop table test2""").collect
-  }
-
-
-  //To check data loading is success when loading from Carbon Table using ‘SINGLE_PASS’=TRUE
-  test("Loading-004-01-01-01_001-TC_061", Include) {
-     sql(s"""create table test1(imei string,AMSize string,channelsId string,ActiveCountry string, Activecity string,gamePointId double,deviceInformationId double,productionDate Timestamp,deliveryDate timestamp,deliverycharge double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('table_blocksize'='1')""").collect
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/vardhandaterestruct.csv' INTO TABLE test1 OPTIONS('DELIMITER'=',', 'QUOTECHAR'= '"','SINGLE_PASS'='TRUE', 'FILEHEADER'= 'imei,deviceInformationId,AMSize,channelsId,ActiveCountry,Activecity,gamePointId,productionDate,deliveryDate,deliverycharge')""").collect
-    checkAnswer(s"""select count(*) from test1""",
-      Seq(Row(99)), "singlepassTestCase_Loading-004-01-01-01_001-TC_061")
-     sql(s"""drop table test1""").collect
-  }
-
-
-  //Verifying load data with single Pass true and BAD_RECORDS_ACTION= ='FAIL
-  test("Loading-004-01-01-01_001-TC_067", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-    intercept[Exception] {
-      sql(s"""
-             | CREATE TABLE uniqdata(
-             | shortField SHORT,
-             | booleanField BOOLEAN,
-             | intField INT,
-             | bigintField LONG,
-             | doubleField DOUBLE,
-             | stringField STRING,
-             | decimalField DECIMAL(18,2),
-             | charField CHAR(5),
-             | floatField FLOAT,
-             | complexData ARRAY<STRING>,
-             | booleanField2 BOOLEAN
-             | )
-             | STORED BY 'carbondata'
-       """.stripMargin)
-
-        .collect
-
-
-      sql(
-        s"""LOAD DATA INPATH  '${TestQueryExecutor
-          .integrationPath}/spark2/src/test/resources/bool/supportBooleanBadRecords.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL','FILEHEADER'='shortField,booleanField,intField,bigintField,doubleField,stringField,timestampField,decimalField,dateField,charField,floatField,complexData,booleanField2','SINGLE_Pass'='true')""".stripMargin)
-        .collect
-      checkAnswer(
-        s"""select count(*) from uniqdata""",
-        Seq(Row(2013)),
-        "singlepassTestCase_Loading-004-01-01-01_001-TC_067")
-  }
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass true and BAD_RECORDS_ACTION= ='REDIRECT'
-  test("Loading-004-01-01-01_001-TC_071", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='REDIRECT','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='true')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_071")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass false and BAD_RECORDS_ACTION= ='REDIRECT'
-  test("Loading-004-01-01-01_001-TC_072", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='REDIRECT','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_072")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass true and BAD_RECORDS_ACTION= ='IGNORE'
-  test("Loading-004-01-01-01_001-TC_073", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='IGNORE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='true')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_073")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass false and BAD_RECORDS_ACTION= ='IGNORE'
-  test("Loading-004-01-01-01_001-TC_074", Include) {
-    sql(s"""drop table if exists uniqdata""").collect
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='IGNORE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_074")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass true and BAD_RECORDS_ACTION= ='FORCE'
-  test("Loading-004-01-01-01_001-TC_075", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='true')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_075")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass false and BAD_RECORDS_ACTION= ='FORCE'
-  test("Loading-004-01-01-01_001-TC_076", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_076")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass false and 'BAD_RECORDS_LOGGER_ENABLE'='TRUE',
-  test("Loading-004-01-01-01_001-TC_077", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_077")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass false and 'BAD_RECORDS_LOGGER_ENABLE'='FALSE',
-  test("Loading-004-01-01-01_001-TC_078", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_078")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass true and 'BAD_RECORDS_LOGGER_ENABLE'='TRUE',
-  test("Loading-004-01-01-01_001-TC_079", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'""").collect
-
-
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/2000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1','SINGLE_Pass'='true')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(2013)), "singlepassTestCase_Loading-004-01-01-01_001-TC_079")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass true, NO_INVERTED_INDEX, and dictionary_exclude
-  test("Loading-004-01-01-01_001-TC_080", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String, DOB timestamp) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('NO_INVERTED_INDEX'='CUST_NAME','dictionary_exclude'='CUST_NAME')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/10_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,DOB','SINGLE_Pass'='true')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(10)), "singlepassTestCase_Loading-004-01-01-01_001-TC_080")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single Pass true, NO_INVERTED_INDEX and dictionary_include a measure
-  test("Loading-004-01-01-01_001-TC_081", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String, DOB timestamp) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID','NO_INVERTED_INDEX'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/10_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,DOB','SINGLE_Pass'='true')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(10)), "singlepassTestCase_Loading-004-01-01-01_001-TC_081")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single pass=false and column dictionary path
-  test("Loading-004-01-01-01_001-TC_084", Include) {
-    dropTable("uniqdata")
-    intercept[Exception] {
-      sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String, DOB timestamp) STORED BY 'org.apache.carbondata.format'""")
-
-        .collect
-      sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/10_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL','FILEHEADER'='CUST_ID,CUST_NAME,DOB','SINGLE_PASS'='false','COLUMNDICT'='CUST_NAME:$resourcesPath/Data/singlepass/data/cust_name.txt')""")
-        .collect
-      checkAnswer(
-        s"""select count(*) from uniqdata""",
-        Seq(Row(10)),
-        "singlepassTestCase_Loading-004-01-01-01_001-TC_084")
-    }
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying load data with single pass=true and column dictionary path
-  test("Loading-004-01-01-01_001-TC_085", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String, DOB timestamp) STORED BY 'org.apache.carbondata.format'""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/10_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL','FILEHEADER'='CUST_ID,CUST_NAME,DOB','SINGLE_PASS'='true','COLUMNDICT'='CUST_NAME:$resourcesPath/Data/singlepass/data/cust_name.txt')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(10)), "singlepassTestCase_Loading-004-01-01-01_001-TC_085")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying single pass false with all dimensions as dictionary_exclude and dictionary_include
-  test("Loading-004-01-01-01_001-TC_088", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String, DOB timestamp) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='CUST_NAME','DICTIONARY_INCLUDE'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/10_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL','FILEHEADER'='CUST_ID,CUST_NAME,DOB','SINGLE_PASS'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(10)), "singlepassTestCase_Loading-004-01-01-01_001-TC_088")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-
-  //Verifying single pass true with all dimensions as dictionary_exclude and dictionary_include
-  test("Loading-004-01-01-01_001-TC_089", Include) {
-    dropTable("uniqdata")
-     sql(s"""CREATE TABLE if not exists uniqdata (CUST_ID int,CUST_NAME String, DOB timestamp) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='CUST_NAME','DICTIONARY_INCLUDE'='CUST_ID')""").collect
-   sql(s"""LOAD DATA INPATH  '$resourcesPath/Data/singlepass/data/10_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL','FILEHEADER'='CUST_ID,CUST_NAME,DOB','SINGLE_PASS'='false')""").collect
-    checkAnswer(s"""select count(*) from uniqdata""",
-      Seq(Row(10)), "singlepassTestCase_Loading-004-01-01-01_001-TC_089")
-     sql(s"""drop table uniqdata""").collect
-  }
-
-  val prop = CarbonProperties.getInstance()
-  val p1 = prop.getProperty("carbon.enable.auto.load.merge", CarbonCommonConstants.DEFAULT_ENABLE_AUTO_LOAD_MERGE)
-
-  override protected def beforeAll() {
-    // Adding new properties
-    prop.addProperty("carbon.enable.auto.load.merge", "true")
-  }
-
-  override def afterAll: Unit = {
-    //Reverting to old
-    prop.addProperty("carbon.enable.auto.load.merge", p1)
-  }
-       
-}
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnExcudeDictTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnExcudeDictTestCase.scala
index 2f531d7..6d05db2 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnExcudeDictTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnExcudeDictTestCase.scala
@@ -34,7 +34,7 @@
   //create table with no dictionary sort_columns
   test("Sortcolumn-001_TC001", Include) {
     sql(s"""drop table if exists sorttable1""").collect
-    sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno','sort_columns'='empno')""").collect
+    sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select empno from sorttable1""").collect
 
@@ -44,7 +44,7 @@
 
   //create table with direct-dictioanry sort_columns
   test("Sortcolumn-001_TC003", Include) {
-    sql(s"""CREATE TABLE sorttable3 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj')""").collect
+    sql(s"""CREATE TABLE sorttable3 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='doj')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable3 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable3""").collect
 
@@ -54,7 +54,7 @@
 
   //create table with multi-sort_columns and data loading with offheap safe
   test("Sortcolumn-001_TC004", Include) {
-    sql(s"""CREATE TABLE sorttable4_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','sort_columns'='workgroupcategory, empname')""").collect
+    sql(s"""CREATE TABLE sorttable4_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable4_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select workgroupcategory, empname from sorttable4_offheap_safe""").collect
 
@@ -64,7 +64,7 @@
 
   //create table with multi-sort_columns and data loading with offheap and unsafe sort
   test("Sortcolumn-001_TC005", Include) {
-    sql(s"""CREATE TABLE sorttable4_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','sort_columns'='workgroupcategory, empname')""").collect
+    sql(s"""CREATE TABLE sorttable4_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable4_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select workgroupcategory, empname from sorttable4_offheap_unsafe""").collect
 
@@ -74,7 +74,7 @@
 
   //create table with multi-sort_columns and data loading with offheap and inmemory sort
   test("Sortcolumn-001_TC006", Include) {
-    sql(s"""CREATE TABLE sorttable4_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','sort_columns'='workgroupcategory, empname')""").collect
+    sql(s"""CREATE TABLE sorttable4_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable4_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select workgroupcategory, empname from sorttable4_offheap_inmemory""").collect
 
@@ -84,7 +84,7 @@
 
   //create table with multi-sort_columns and data loading with heap
   test("Sortcolumn-001_TC007", Include) {
-    sql(s"""CREATE TABLE sorttable4_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','sort_columns'='workgroupcategory, empname')""").collect
+    sql(s"""CREATE TABLE sorttable4_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable4_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select workgroupcategory, empname from sorttable4_heap_safe""").collect
 
@@ -94,7 +94,7 @@
 
   //create table with multi-sort_columns and data loading with heap and unsafe sort
   test("Sortcolumn-001_TC008", Include) {
-    sql(s"""CREATE TABLE sorttable4_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','sort_columns'='workgroupcategory, empname')""").collect
+    sql(s"""CREATE TABLE sorttable4_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable4_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select workgroupcategory, empname from sorttable4_heap_unsafe""").collect
 
@@ -104,7 +104,7 @@
 
   //create table with multi-sort_columns and data loading with heap and inmemory sort
   test("Sortcolumn-001_TC009", Include) {
-    sql(s"""CREATE TABLE sorttable4_heap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','sort_columns'='workgroupcategory, empname')""").collect
+    sql(s"""CREATE TABLE sorttable4_heap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable4_heap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select workgroupcategory, empname from sorttable4_heap_inmemory""").collect
 
@@ -122,7 +122,7 @@
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE origintable2 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE origintable2 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""alter table origintable2 compact 'minor'""").collect
-    sql(s"""CREATE TABLE sorttable5 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno','sort_columns'='empno')""").collect
+    sql(s"""CREATE TABLE sorttable5 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable5 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable5 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable5 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
@@ -137,7 +137,7 @@
   //filter on sort_columns include no-dictionary
   test("Sortcolumn-001_TC011", Include) {
     sql(s"""drop table if exists sorttable6""").collect
-    sql(s"""CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','dictionary_include'='doj','sort_columns'='workgroupcategory, doj, empname')""").collect
+    sql(s"""CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='workgroupcategory, doj, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable6 where workgroupcategory = 1""").collect
 
@@ -147,7 +147,7 @@
 
   //filter on sort_columns include direct-dictionary
   test("Sortcolumn-001_TC012", Include) {
-    sql(s"""CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','dictionary_include'='doj','sort_columns'='workgroupcategory, doj, empname')""").collect
+    sql(s"""CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='workgroupcategory, doj, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable6 where doj = '2007-01-17 00:00:00'""").collect
 
@@ -158,7 +158,7 @@
   //filter on sort_columns include dictioanry
   test("Sortcolumn-001_TC013", Include) {
     sql(s"""drop table if exists sorttable6""").collect
-    sql(s"""CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='workgroupcategory','dictionary_include'='doj','sort_columns'='workgroupcategory, doj, empname')""").collect
+    sql(s"""CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='workgroupcategory, doj, empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable6 where empname = 'madhan'""").collect
 
@@ -179,7 +179,7 @@
   //unsorted table creation, query data loading with heap and safe sort config with order by
   test("Sortcolumn-001_TC015", Include) {
     sql(s"""drop table if exists unsortedtable_heap_safe""").collect
-    sql(s"""CREATE TABLE unsortedtable_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_heap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_heap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_heap_safe order by empno""").collect
 
@@ -189,7 +189,7 @@
 
   //unsorted table creation, query and data loading with heap and unsafe sort config
   test("Sortcolumn-001_TC016", Include) {
-    sql(s"""CREATE TABLE unsortedtable_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_heap_unsafe where empno = 11""").collect
 
@@ -200,7 +200,7 @@
   //unsorted table creation, query and data loading with heap and unsafe sort config with order by
   test("Sortcolumn-001_TC017", Include) {
     sql(s"""drop table if exists unsortedtable_heap_unsafe""").collect
-    sql(s"""CREATE TABLE unsortedtable_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_heap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_heap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_heap_unsafe order by empno""").collect
 
@@ -210,7 +210,7 @@
 
   //unsorted table creation, query and data loading with offheap and safe sort config
   test("Sortcolumn-001_TC018", Include) {
-    sql(s"""CREATE TABLE unsortedtable_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_offheap_safe where empno = 11""").collect
 
@@ -220,7 +220,7 @@
 
   //unsorted table creation, query and data loading with offheap and safe sort config with order by
   test("Sortcolumn-001_TC019", Include) {
-    sql(s"""CREATE TABLE unsortedtable_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_offheap_safe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_offheap_safe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_offheap_safe order by empno""").collect
 
@@ -230,7 +230,7 @@
 
   //unsorted table creation, query and data loading with offheap and unsafe sort config
   test("Sortcolumn-001_TC020", Include) {
-    sql(s"""CREATE TABLE unsortedtable_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_offheap_unsafe where empno = 11""").collect
 
@@ -240,7 +240,7 @@
 
   //unsorted table creation, query and data loading with offheap and unsafe sort config with order by
   test("Sortcolumn-001_TC021", Include) {
-    sql(s"""CREATE TABLE unsortedtable_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_offheap_unsafe (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_offheap_unsafe OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_offheap_unsafe order by empno""").collect
 
@@ -250,7 +250,7 @@
 
   //unsorted table creation, query and data loading with offheap and inmemory sort config
   test("Sortcolumn-001_TC022", Include) {
-    sql(s"""CREATE TABLE unsortedtable_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_offheap_inmemory where empno = 11""").collect
 
@@ -260,7 +260,7 @@
 
   //unsorted table creation, query and data loading with offheap and inmemory sort config with order by
   test("Sortcolumn-001_TC023", Include) {
-    sql(s"""CREATE TABLE unsortedtable_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','sort_columns'='')""").collect
+    sql(s"""CREATE TABLE unsortedtable_offheap_inmemory (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE unsortedtable_offheap_inmemory OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from unsortedtable_offheap_inmemory order by empno""").collect
 
@@ -271,42 +271,19 @@
   //create table with dictioanry_exclude sort_columns
   test("Sortcolumn-001_TC024", Include) {
     sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,deptno,workgroupcategory,projectcode,attendance,utilization,salary','dictionary_include'='doj','dictionary_exclude'='empname','sort_columns'='empname')""").collect
+    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
     sql(s"""drop table if exists sorttable""").collect
   }
 
-
-  //create table with dictionary_include,  sort_columns
-  test("Sortcolumn-001_TC025", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='doj','sort_columns'='doj')""").collect
-    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-    sql(s"""drop table if exists sorttable""").collect
-  }
-
-
-  //create table with dictionary_include, dictioanry_exclude sort_columns
-  test("Sortcolumn-001_TC026", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empname,doj','sort_columns'='doj')""").collect
-    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-    sql(s"""drop table if exists sorttable""").collect
-  }
-
-
   //create table with alter table and sort_columns with dimension
   test("Sortcolumn-001_TC027", Include) {
     sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj')""").collect
+    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='doj')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""alter table sorttable add columns(newField String) tblproperties('dictionary_include'='newField')""").collect
+    sql(s"""alter table sorttable add columns(newField String) """).collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataString.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -317,42 +294,19 @@
   //create table with alter table and sort_columns with measure
   test("Sortcolumn-001_TC028", Include) {
     sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj')""").collect
+    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='doj')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""alter table sorttable add columns(newField Int) tblproperties('dictionary_include'='newField')""").collect
+    sql(s"""alter table sorttable add columns(newField Int) """).collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataInt.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
     sql(s"""drop table if exists sorttable""").collect
   }
 
-
-  //create table with dictionary_include ,no_inverted_index and sort_columns
-  test("Sortcolumn-001_TC030", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj','no_inverted_index'='doj')""").collect
-    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-    sql(s"""drop table if exists sorttable""").collect
-  }
-
-
-  //create table with dictionary_include ,no_inverted_index and sort_columns with measure
-  test("Sortcolumn-001_TC031", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno','sort_columns'='empno','no_inverted_index'='empno')""").collect
-    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-    sql(s"""drop table if exists sorttable""").collect
-  }
-
-
   //test sort_column for different order of column name
   test("Sortcolumn-001_TC032", Include) {
     sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno,workgroupcategory','dictionary_include'='doj','sort_columns'='empname,empno,workgroupcategory,doj')""").collect
+    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname,empno,workgroupcategory,doj')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -363,7 +317,7 @@
   //default behavior if sort_column not provided
   test("Sortcolumn-001_TC033", Include) {
     sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno')""").collect
+    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' """).collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -374,7 +328,7 @@
   //test sort_column for alter table
   test("Sortcolumn-001_TC035", Include) {
     sql(s"""drop table if exists sorttable""").collect
-    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj')""").collect
+    sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='doj')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""alter table sorttable drop columns(doj)""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataDrop.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
@@ -388,9 +342,9 @@
   test("Sortcolumn-001_TC037", Include) {
     sql(s"""drop table if exists sorttable""").collect
     sql(s"""drop table if exists sorttable1""").collect
-    sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno','sort_columns'='empno')""").collect
+    sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""alter table sorttable1 add columns(newField Float) tblproperties('DICTIONARY_INCLUDE'='newField')""").collect
+    sql(s"""alter table sorttable1 add columns(newField Float) """).collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataFloat.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable1""").collect
 
@@ -402,9 +356,9 @@
   test("Sortcolumn-001_TC038", Include) {
     sql(s"""drop table if exists sorttable""").collect
     sql(s"""drop table if exists sorttable1""").collect
-    sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empno','sort_columns'='empno')""").collect
+    sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')""").collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""alter table sorttable1 add columns(newField decimal) tblproperties('dictionary_include'='newField')""").collect
+    sql(s"""alter table sorttable1 add columns(newField decimal) """).collect
     sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataDecimal.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable1""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnTestCase.scala
index 3c554ff..f299073 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/SortColumnTestCase.scala
@@ -281,42 +281,19 @@
   //create table with dictioanry_exclude sort_columns
   test("Sortcolumn-001_TC024", Include) {
     sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_exclude'='empname','sort_columns'='empname')""").collect
+     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname')""").collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
      sql(s"""drop table if exists sorttable""").collect
   }
 
-
-  //create table with dictionary_include,  sort_columns
-  test("Sortcolumn-001_TC025", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj')""").collect
-   sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-     sql(s"""drop table if exists sorttable""").collect
-  }
-
-
-  //create table with dictionary_include, dictioanry_exclude sort_columns
-  test("Sortcolumn-001_TC026", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','dictionary_exclude'='empname','sort_columns'='doj')""").collect
-   sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-     sql(s"""drop table if exists sorttable""").collect
-  }
-
-
   //create table with alter table and sort_columns with dimension
   test("Sortcolumn-001_TC027", Include) {
     sql(s"""drop table if exists sorttable""").collect
      sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='doj')""").collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-   sql(s"""alter table sorttable add columns(newField String) tblproperties('dictionary_include'='newField')""").collect
+   sql(s"""alter table sorttable add columns(newField String)  """).collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataString.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -329,7 +306,7 @@
     sql(s"""drop table if exists sorttable""").collect
      sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='doj')""").collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-   sql(s"""alter table sorttable add columns(newField Int) tblproperties('dictionary_include'='newField')""").collect
+   sql(s"""alter table sorttable add columns(newField Int)  """).collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataInt.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -347,33 +324,10 @@
      sql(s"""drop table if exists sorttable""").collect
   }
 
-
-  //create table with dictionary_include ,no_inverted_index and sort_columns
-  test("Sortcolumn-001_TC030", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='doj','sort_columns'='doj','no_inverted_index'='doj')""").collect
-   sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-     sql(s"""drop table if exists sorttable""").collect
-  }
-
-
-  //create table with dictionary_include ,no_inverted_index and sort_columns with measure
-  test("Sortcolumn-001_TC031", Include) {
-    sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='empno','sort_columns'='empno','no_inverted_index'='empno')""").collect
-   sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-    sql(s"""select doj from sorttable""").collect
-
-     sql(s"""drop table if exists sorttable""").collect
-  }
-
-
   //test sort_column for different order of column name
   test("Sortcolumn-001_TC032", Include) {
     sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='empno','sort_columns'='empname,empno,workgroupcategory,doj')""").collect
+     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties( 'sort_columns'='empname,empno,workgroupcategory,doj')""").collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -384,7 +338,7 @@
   //default behavior if sort_column not provided
   test("Sortcolumn-001_TC033", Include) {
     sql(s"""drop table if exists sorttable""").collect
-     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='empno')""").collect
+     sql(s"""CREATE TABLE sorttable (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select doj from sorttable""").collect
 
@@ -411,7 +365,7 @@
     sql(s"""drop table if exists sorttable1""").collect
      sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')""").collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-   sql(s"""alter table sorttable1 add columns(newField Float) tblproperties('DICTIONARY_INCLUDE'='newField')""").collect
+   sql(s"""alter table sorttable1 add columns(newField Float)  """).collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataFloat.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable1""").collect
 
@@ -425,7 +379,7 @@
     sql(s"""drop table if exists sorttable1""").collect
      sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empno')""").collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/data.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
-   sql(s"""alter table sorttable1 add columns(newField decimal) tblproperties('dictionary_include'='newField')""").collect
+   sql(s"""alter table sorttable1 add columns(newField decimal)  """).collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataDecimal.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable1""").collect
 
@@ -437,7 +391,7 @@
   test("Sortcolumn-001_TC039", Include) {
     sql(s"""drop table if exists sorttable""").collect
     sql(s"""drop table if exists sorttable1""").collect
-     sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int,newField decimal) STORED BY 'org.apache.carbondata.format' tblproperties('DICTIONARY_INCLUDE'='empno')""").collect
+     sql(s"""CREATE TABLE sorttable1 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int,newField decimal) STORED BY 'org.apache.carbondata.format' """).collect
    sql(s"""LOAD DATA local inpath '$resourcesPath/Data/sortcolumns/dataDecimal.csv' INTO TABLE sorttable1 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""").collect
     sql(s"""select * from sorttable1""").collect
 
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala
index 855b560..fd4b432 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala
@@ -69,22 +69,6 @@
 
   }
 
-  //Verify exception while Creating a partition table with DICTIONARY_INCLUDE
-  test("Standard-Partition_TC005", Include) {
-    sql(s"""drop table if exists partition_table_string""")
-    intercept[Exception] {
-      sql(s"""CREATE TABLE partition_table_string(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestampField TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT, complexData ARRAY<STRING> ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='stringField')""")
-    }
-    sql(s"""drop table if exists partition_table_string""")
-  }
-
-  //Creating a partition table with DICTIONARY_EXCLUDE
-  test("Standard-Partition_TC006", Include) {
-    sql(s"""drop table if exists partition_table_string""")
-    sql(s"""CREATE TABLE partition_table_string(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestampField TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT, complexData ARRAY<STRING> ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='stringField')""")
-    sql(s"""drop table if exists partition_table_string""")
-  }
-
   //Verify exception if datatype is not provided with partition column
   test("Standard-Partition_TC007", Include) {
     sql(s"""drop table if exists uniqdata""")
@@ -167,22 +151,6 @@
     checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(0)))
   }
 
-  //Loading data into a partitioned table with SINGLE_PASS=TRUE
-  test("Standard-Partition_TC018", Include) {
-    sql(s"""drop table if exists partition_table""")
-    sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
-    sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('SINGLE_PASS'='TRUE')""")
-    checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
-  }
-
-  //Loading data into a partitioned table with SINGLE_PASS=FALSE
-  test("Standard-Partition_TC019", Include) {
-    sql(s"""drop table if exists partition_table""")
-    sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
-    sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('SINGLE_PASS'='FALSE')""")
-    checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
-  }
-
   //Verify load with Standard Partition
   test("Standard-Partition_TC020", Include) {
     sql(s"""drop table if exists uniqdata""")
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TestPartitionWithGlobalSort.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TestPartitionWithGlobalSort.scala
index 31ce5fa..622175b 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TestPartitionWithGlobalSort.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/TestPartitionWithGlobalSort.scala
@@ -204,7 +204,7 @@
     sql(
       s"""Create table s ( s short, f float,l long,d double,ch char(10),vch varchar(10),num int,time timestamp,
          |dt date,name string) partitioned by (dec decimal(30,15)) stored by 'carbondata'
-         |TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT','DICTIONARY_EXCLUDE'='name')""".stripMargin)
+         |TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')""".stripMargin)
     sql("show partitions s")
     sql(s"""alter table s rename to partition2""")
     intercept[Exception] {
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/datasource/SparkCarbonDataSourceTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/datasource/SparkCarbonDataSourceTestCase.scala
index abdced7..aa271c4 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/datasource/SparkCarbonDataSourceTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/datasource/SparkCarbonDataSourceTestCase.scala
@@ -745,8 +745,7 @@
 
   test("test read using old data") {
     val store = new StoreCreator(new File(warehouse1).getAbsolutePath,
-      new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath,
-      false)
+      new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath)
     store.createCarbonStore()
     FileFactory
       .deleteAllFilesOfDir(new File(warehouse1 + "/testdb/testtable/Fact/Part0/Segment_0/0"))
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala
index 7d93f15..b32e45d 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/register/TestRegisterCarbonTable.scala
@@ -167,7 +167,7 @@
       sql("refresh table carbontable")
     }
     sql("Alter table carbontable add columns(c4 string) " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='c4', 'DEFAULT.VALUE.c4'='def')")
+        "TBLPROPERTIES('DEFAULT.VALUE.c4'='def')")
     checkAnswer(
       sql("""select c1,c2,c3,c5,c4 from carbon.carbontable"""),
       Seq(Row("a", 1, "aa", "aaa", "def"), Row("b", 1, "bb", "bbb", "def"))
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
index 1d14d06..09fcc1d 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
@@ -31,7 +31,6 @@
 
   val suites = new AlterTableTestCase ::
                new BadRecordTestCase ::
-               new ColumndictTestCase ::
                new CreateTableAsSelectTestCase ::
                new DataLoadingTestCase ::
                new DataLoadingV3TestCase ::
@@ -43,13 +42,10 @@
                new QueriesBasicTestCase ::
                new QueriesBVATestCase ::
                new QueriesCompactionTestCase ::
-               new QueriesExcludeDictionaryTestCase ::
-               new QueriesIncludeDictionaryTestCase ::
                new QueriesNormalTestCase ::
                new QueriesRangeFilterTestCase ::
                new QueriesSparkBlockDistTestCase ::
                new ShowLoadsTestCase ::
-               new SinglepassTestCase ::
                new SortColumnTestCase ::
                new TableCommentAlterTableTestCase ::
                new TimestamptypesTestCase ::
@@ -98,7 +94,6 @@
 class SDVSuites2 extends Suites with BeforeAndAfterAll {
 
   val suites = new QueriesBVATestCase ::
-               new QueriesExcludeDictionaryTestCase ::
                new DataLoadingIUDTestCase :: Nil
 
   override val nestedSuites = suites.toIndexedSeq
@@ -139,18 +134,15 @@
                new OffheapQuery2TestCase ::
                new OffheapSort1TestCase ::
                new ShowLoadsTestCase ::
-               new SinglepassTestCase ::
                new SortColumnTestCase ::
                new TimestamptypesTestCase ::
                new V3offheapvectorTestCase ::
                new Vector1TestCase ::
                new Vector2TestCase ::
                new QueriesNormalTestCase ::
-               new ColumndictTestCase ::
                new QueriesRangeFilterTestCase ::
                new QueriesSparkBlockDistTestCase ::
                new DataLoadingV3TestCase ::
-               new QueriesIncludeDictionaryTestCase ::
                new TestRegisterCarbonTable ::
                new TableCommentAlterTableTestCase ::
                new StandardPartitionTestCase ::
diff --git a/integration/spark-common-test/src/test/resources/dict.txt b/integration/spark-common-test/src/test/resources/dict.txt
deleted file mode 100644
index 64ba47f..0000000
--- a/integration/spark-common-test/src/test/resources/dict.txt
+++ /dev/null
@@ -1 +0,0 @@
-12154
\ No newline at end of file
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/bigdecimal/TestDimensionWithDecimalDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/bigdecimal/TestDimensionWithDecimalDataType.scala
index eed2f32..33fe01a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/bigdecimal/TestDimensionWithDecimalDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/bigdecimal/TestDimensionWithDecimalDataType.scala
@@ -32,7 +32,7 @@
     sql("drop table if exists hiveTable")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-    sql("CREATE TABLE IF NOT EXISTS carbonTable (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary Decimal(19,2))STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='salary')")
+    sql("CREATE TABLE IF NOT EXISTS carbonTable (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary Decimal(19,2))STORED BY 'org.apache.carbondata.format' ")
     sql("create table if not exists hiveTable(ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary Decimal(19,2))row format delimited fields terminated by ','")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalDataWithHeader.csv' into table carbonTable")
     sql(s"LOAD DATA local inpath '$resourcesPath/decimalDataWithoutHeader.csv' INTO table hiveTable")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
index 812aab5..7550581 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
@@ -172,63 +172,6 @@
         assert(true)
     }
 
-
-    test("Unsupport DICTIONARY_INCLUDE for binary") {
-
-        sql("DROP TABLE IF EXISTS binaryTable")
-        val exception = intercept[MalformedCarbonCommandException] {
-            sql(
-                """
-                  | CREATE TABLE binaryTable(
-                  |     id int,
-                  |     name string,
-                  |     city string,
-                  |     age int,
-                  |     binaryField binary)
-                  | STORED BY 'carbondata'
-                  | tblproperties('dictionary_include'='binaryField')
-                """.stripMargin)
-        }
-        assert(exception.getMessage.contains(
-            "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryField"))
-    }
-
-    test("Unsupport DICTIONARY_INCLUDE for binary, multiple column") {
-
-        sql("DROP TABLE IF EXISTS binaryTable")
-        val exception = intercept[MalformedCarbonCommandException] {
-            sql(
-                """
-                  | CREATE TABLE binaryTable(
-                  |     id int,
-                  |     name string,
-                  |     city string,
-                  |     age int,
-                  |     binaryField binary)
-                  | STORED BY 'carbondata'
-                  | tblproperties('dictionary_include'='name,binaryField')
-                """.stripMargin)
-        }
-        assert(exception.getMessage.contains(
-            "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryField"))
-    }
-
-    test("Supports DICTIONARY_EXCLUDE for binary") {
-        sql("DROP TABLE IF EXISTS binaryTable")
-        sql(
-            """
-              | CREATE TABLE binaryTable(
-              |     id int,
-              |     name string,
-              |     city string,
-              |     age int,
-              |     binaryField binary)
-              | STORED BY 'org.apache.carbondata.format'
-              | tblproperties('DICTIONARY_EXCLUDE'='binaryField')
-            """.stripMargin)
-        assert(true)
-    }
-
     test("Unsupport inverted_index for binary") {
         sql("DROP TABLE IF EXISTS binaryTable")
         val exception = intercept[MalformedCarbonCommandException] {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala
index 4b3d283..c568a99 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCompactionComplexType.scala
@@ -1024,8 +1024,7 @@
         "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
         "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
         "double,contractNumber double) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdata.csv' INTO table " +
@@ -1085,8 +1084,7 @@
       "STRUCT_OF_ARRAY struct<ID:int,CHECK_DATE:string,SNo:array<int>,sal1:array<double>," +
       "state:array<string>," +
       "date1:array<string>>,CARD_COUNT int,DEBIT_COUNT int,CREDIT_COUNT int, DEPOSIT double, " +
-      "HQ_DEPOSIT double) STORED BY 'carbondata'" +
-      "TBLPROPERTIES('DICTIONARY_INCLUDE'='STRUCT_OF_ARRAY,DEPOSIT,HQ_DEPOSIT')")
+      "HQ_DEPOSIT double) STORED BY 'carbondata'")
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/structofarray.csv' INTO TABLE compactComplex OPTIONS" +
       s"('DELIMITER'=',','QUOTECHAR'='\'," +
@@ -1122,10 +1120,7 @@
         | number string,
         | structfield struct<a:array<int> ,b:int>
         | )
-        | stored by 'carbondata'
-        | TBLPROPERTIES(
-        | 'DICTIONARY_INCLUDE'='name,age,number,structfield'
-        | )
+        | stored as carbondata
       """.stripMargin)
     sql("INSERT into compactComplex values('man',25,'222',named_struct('a', array(1000,2000), 'b', 1))")
     sql("INSERT into compactComplex values('can',24,'333',named_struct('a', array(1000,2000), 'b', 2))")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
index bfc1935..2dbae36 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexDataType.scala
@@ -414,9 +414,7 @@
 
   test("test StructofArray pushdown") {
     sql("DROP TABLE IF EXISTS table1")
-    sql(
-      "create table table1 (person Struct<detail:string,ph:array<int>>) stored by " +
-      "'carbondata' tblproperties('dictionary_include'='person')")
+    sql("create table table1 (person Struct<detail:string,ph:array<int>>) stored by 'carbondata' ")
     sql("insert into table1 values(named_struct('detail', 'abc', 'ph', array(2)))")
     sql("select person from table1").show(false)
     sql("select person.detail, person.ph[0] from table1").show(false)
@@ -636,9 +634,7 @@
     sql("DROP TABLE IF EXISTS table1")
     sql(
       "create table table1 (roll int,a struct<b:int,c:string,d:int,e:string,f:struct<g:int," +
-      "h:string,i:int>,j:int>) stored " +
-      "by " +
-      "'carbondata' tblproperties('dictionary_include'='a')")
+      "h:string,i:int>,j:int>) stored by 'carbondata' ")
     sql("insert into table1 values(1,named_struct('b', 1, 'c', 'abc', 'd', 2, 'e', 'efg', 'f', named_struct('g', 3, 'h', 'mno', 'i', 4), 'j', 5))")
     sql("insert into table1 values(2,named_struct('b', 1, 'c', 'abc', 'd', 2, 'e', 'efg', 'f', named_struct('g', 3, 'h', 'mno', 'i', 4), 'j', 5))")
     sql("insert into table1 values(3,named_struct('b', 1, 'c', 'abc', 'd', 2, 'e', 'efg', 'f', named_struct('g', 3, 'h', 'mno', 'i', 4), 'j', 5))")
@@ -827,62 +823,6 @@
     assertResult("Cannot use struct<b:int> for partition column;")(structException.getMessage)
   }
 
-  test("test block dictionary exclude for child column") {
-    sql("DROP TABLE IF EXISTS table1")
-    sql(
-      "create table table1 (roll int,a struct<b:int,c:string,d:int,e:string,f:struct<g:int," +
-      "h:string,i:int>,j:int>) stored " +
-      "by " +
-      "'carbondata' tblproperties('dictionary_exclude'='a')")
-    sql("insert into table1 values(1,named_struct('b', 1, 'c', 'abc', 'd', 2, 'e', 'efg', 'f', named_struct('g', 3, 'h', 'mno', 'i', 4), 'j', 5))")
-    checkAnswer(sql("select a.b from table1"), Seq(Row(1)))
-    sql("DROP TABLE IF EXISTS table1")
-    val structException = intercept[MalformedCarbonCommandException](
-    sql(
-      "create table table1 (roll int,a struct<b:int,c:string,d:int,e:string,f:struct<g:int," +
-      "h:string,i:int>,j:int>) stored " +
-      "by " +
-      "'carbondata' tblproperties('dictionary_exclude'='a.b')"))
-    assertResult(
-      "DICTIONARY_EXCLUDE column: a.b does not exist in table or unsupported for complex child " +
-      "column. Please check the create table statement.")(
-      structException.getMessage)
-    sql("DROP TABLE IF EXISTS table1")
-    val arrayException = intercept[MalformedCarbonCommandException](
-      sql(
-        "create table table1 (roll int,a array<int>) stored " +
-        "by " +
-        "'carbondata' tblproperties('dictionary_exclude'='a[0]')"))
-    assertResult(
-      "DICTIONARY_EXCLUDE column: a[0] does not exist in table or unsupported for complex child " +
-      "column. Please check the create table statement.")(
-      arrayException.getMessage)
-  }
-
-  test("test block dictionary include for child column") {
-    sql("DROP TABLE IF EXISTS table1")
-    val structException = intercept[MalformedCarbonCommandException](
-      sql(
-        "create table table1 (roll int,a struct<b:int,c:string,d:int,e:string,f:struct<g:int," +
-        "h:string,i:int>,j:int>) stored " +
-        "by " +
-        "'carbondata' tblproperties('dictionary_include'='a.b')"))
-    assertResult(
-      "DICTIONARY_INCLUDE column: a.b does not exist in table or unsupported for complex child " +
-      "column. Please check the create table statement.")(
-      structException.getMessage)
-    sql("DROP TABLE IF EXISTS table1")
-    val arrayException = intercept[MalformedCarbonCommandException](
-      sql(
-        "create table table1 (roll int,a array<int>) stored " +
-        "by " +
-        "'carbondata' tblproperties('dictionary_include'='a[0]')"))
-    assertResult(
-      "DICTIONARY_INCLUDE column: a[0] does not exist in table or unsupported for complex child " +
-      "column. Please check the create table statement.")(
-      arrayException.getMessage)
-  }
-
   test("test complex datatype double for encoding") {
     sql("DROP TABLE IF EXISTS table1")
     sql(
@@ -936,8 +876,7 @@
   test("decimal with two level struct type") {
     sql("DROP TABLE IF EXISTS test")
     sql(
-      "create table test(id int,a struct<c:struct<d:decimal(20,10)>>) stored by 'carbondata' " +
-      "tblproperties('dictionary_include'='a')")
+      "create table test(id int,a struct<c:struct<d:decimal(20,10)>>) stored by 'carbondata' ")
     checkExistence(sql("desc test"),true,"struct<c:struct<d:decimal(20,10)>>")
     checkExistence(sql("describe formatted test"),true,"struct<c:struct<d:decimal(20,10)>>")
     sql("insert into test values(1, named_struct('c', named_struct('d', 3999.999)))")
@@ -948,20 +887,18 @@
     sql("DROP TABLE IF EXISTS test")
     sql(
       "create table test(id int,a struct<b:int,c:int>, d struct<e:int,f:int>, d1 struct<e1:int," +
-      "f1:int>) stored by 'carbondata' tblproperties('dictionary_include'='d1')")
+      "f1:int>) stored by 'carbondata' ")
     sql("insert into test values(1, named_struct('b', 2, 'c', 3), named_struct('e', 4, 'f', 5), named_struct('e1', 6, 'f1', 7))")
     checkAnswer(sql("select * from test"),Seq(Row(1,Row(2,3),Row(4,5),Row(6,7))))
     sql("DROP TABLE IF EXISTS test")
     sql(
-      "create table test(a array<int>, b array<int>) stored by 'carbondata' tblproperties" +
-      "('dictionary_include'='b')")
+      "create table test(a array<int>, b array<int>) stored by 'carbondata'")
     sql("insert into test values(array(1),array(2)) ")
     checkAnswer(sql("select b[0] from test"),Seq(Row(2)))
     sql("DROP TABLE IF EXISTS test")
     sql(
       "create table test(intval array<array<int>>,str array<array<string>>, bool " +
-      "array<array<boolean>>, sint array<array<short>>, big array<array<bigint>>)  stored by " +
-      "'carbondata' tblproperties('dictionary_include'='bool,sint,big')")
+      "array<array<boolean>>, sint array<array<short>>, big array<array<bigint>>)  stored by 'carbondata' ")
     sql("insert into test values(array(array(1)), array(array('ab')), array(array(true)), array(array(22)), array(array(33))) ")
     checkExistence(sql("select * from test"), true, "33")
   }
@@ -1172,8 +1109,8 @@
     sql(s"""LOAD DATA inpath '${resourcesPath}/complexdata3.csv' INTO table ${tableName}
         options('DELIMITER'='\t','QUOTECHAR'='"','COMMENTCHAR'='#','HEADER'='false',
                 'FILEHEADER'='id,phone,phone_country,phone_province,phone_city,other_phone,other_phone_country,other_phone_province,other_phone_city,call_type,begin_time,begin_hhmm,ds,dss,dur,voice_flag,modela,modelb,modela_pk,modelb_pk,modela_ms,modelb_ms,lang,lang_dec,lang_sc,gender,nlp_sc,tl,vtl,create_time,cdr_create_time,fulltext,tag_label,tag_memo,tag_listen,tag_imp,prop,files',
-                'MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='\\001','COMPLEX_DELIMITER_LEVEL_2'='\\002',
-                'SINGLE_PASS'='TRUE')""")
+                'MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='\\001','COMPLEX_DELIMITER_LEVEL_2'='\\002'
+                )""")
     checkAnswer(sql(s"select count(1) from ${tableName}"), Seq(Row(10)))
     checkAnswer(sql(s"select modela[0][0], modela_ms[0][1] from ${tableName} where id = 'e01a1773-bd37-40be-a1de-d7e74837a281'"),
       Seq(Row(0.0, 0.10781755)))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala
index f08aa20..312f0d8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala
@@ -46,7 +46,7 @@
     try {
       sql("CREATE TABLE doubleComplex (Id int, number double, name string, " +
         "gamePoint array<double>, mac struct<num:double>) " +
-        "STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include' = 'gamePoint,mac')")
+        "STORED BY 'org.apache.carbondata.format' ")
       sql(s"LOAD DATA LOCAL INPATH '$dataPath' INTO TABLE doubleComplex")
       countNum = sql(s"SELECT COUNT(*) FROM doubleComplex").collect
       doubleField = sql("SELECT number FROM doubleComplex SORT BY Id").collect
@@ -64,22 +64,13 @@
     try {
       sql("CREATE TABLE doubleComplex2 (Id int, number double, name string, " +
         "gamePoint array<double>, mac struct<num:double>) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='number,gamePoint,mac')")
+        "STORED BY 'org.apache.carbondata.format' ")
       sql(s"LOAD DATA LOCAL INPATH '$dataPath' INTO TABLE doubleComplex2")
       countNum = sql(s"SELECT COUNT(*) FROM doubleComplex2").collect
       doubleField = sql(s"SELECT number FROM doubleComplex2 SORT BY Id").collect
     } catch {
       case e : Throwable => fail(e)
     }
-    // assert that field 'number' is a dimension
-    val tableIdentifier = new CarbonTableIdentifier(
-      CarbonCommonConstants.DATABASE_DEFAULT_NAME, "doubleComplex2".toLowerCase(), "uniqueid")
-    val carbonTable =
-      CarbonMetadata.getInstance().getCarbonTable(tableIdentifier.getTableUniqueName)
-    val dimExist = carbonTable.getVisibleDimensions().toArray.
-      exists(_.asInstanceOf[CarbonDimension].getColName.equalsIgnoreCase("number"))
-    assertResult(dimExist)(true)
     // assert that load and query is successful
     assertResult(countNum)(Array(Row(3)))
     assertResult(doubleField)(Array(Row(1.5), Row(2.0), Row(3.0)))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index c15917d..4f7947d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -127,24 +127,6 @@
     sql("DROP TABLE load_test")
   }
 
-  test("test data loading into table with Single Pass") {
-    sql("DROP TABLE IF EXISTS load_test_singlepass")
-    sql(""" CREATE TABLE load_test_singlepass(id int, name string, city string, age int)
-        STORED BY 'org.apache.carbondata.format' """)
-    val testData = s"$resourcesPath/sample.csv"
-    try {
-      sql(s"LOAD DATA LOCAL INPATH '$testData' into table load_test_singlepass options ('SINGLE_PASS'='TRUE')")
-    } catch {
-      case ex: Exception =>
-        assert(false)
-    }
-    checkAnswer(
-      sql("SELECT id,name FROM load_test_singlepass where name='eason'"),
-      Seq(Row(2,"eason"))
-    )
-    sql("DROP TABLE load_test_singlepass")
-  }
-
   test("test load data with decimal type and sort intermediate files as 1") {
     sql("drop table if exists carbon_table")
     CarbonProperties.getInstance()
@@ -202,30 +184,6 @@
     sql("drop table if exists load32000bytes")
   }
 
-  test("test load / insert / update with data more than 32000 bytes - dictionary_include") {
-    val testdata = s"$resourcesPath/unicodechar.csv"
-    sql("drop table if exists load32000bytes")
-    sql("create table load32000bytes(name string) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='name')")
-    sql("insert into table load32000bytes select 'aaa'")
-
-    assert(intercept[Exception] {
-      sql(s"load data local inpath '$testdata' into table load32000bytes OPTIONS ('FILEHEADER'='name')")
-    }.getMessage.contains("generate global dictionary failed, Dataload failed, String size cannot exceed 32000 bytes"))
-
-    val source = scala.io.Source.fromFile(testdata, CarbonCommonConstants.DEFAULT_CHARSET)
-    val data = source.mkString
-
-    intercept[Exception] {
-      sql(s"insert into load32000bytes values('$data')")
-    }
-
-    intercept[Exception] {
-      sql(s"update load32000bytes set(name)= ('$data')").show()
-    }
-
-    sql("drop table if exists load32000bytes")
-  }
-
   test("test if stale folders are deleting on data load") {
     sql("drop table if exists stale")
     sql("create table stale(a string) stored by 'carbondata'")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
index 9fbc061..0668b7b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
@@ -227,7 +227,6 @@
          |  ${if (StringUtils.isBlank(columnCompressor)) "" else s"'${CarbonCommonConstants.COMPRESSOR}'='$columnCompressor',"}
          |  ${if (streaming) "" else s"'LONG_STRING_COLUMNS'='longStringField',"}
          |  'SORT_COLUMNS'='stringSortField',
-         |  'DICTIONARY_INCLUDE'='stringDictField',
          |  'local_dictionary_enable'='true',
          |  'local_dictionary_threshold'='10000',
          |  'local_dictionary_include'='stringLocalDictField' ${if (streaming) s", 'STREAMING'='true'" else ""})
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala
deleted file mode 100644
index f2c540c..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.integration.spark.testsuite.dataload
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-/**
-  * Test Class for data loading use one pass
-  *
-  */
-class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    sql("DROP TABLE IF EXISTS table_two_pass")
-    sql("DROP TABLE IF EXISTS table_one_pass")
-    sql("DROP TABLE IF EXISTS table_one_pass_2")
-
-    sql(
-      """
-        |CREATE TABLE table_two_pass (ID int, date Timestamp, country String,
-        |name String, phonetype String, serialname String, salary int)
-        |STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(
-      s"""
-        |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_two_pass
-        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false')
-      """.stripMargin)
-
-    sql(
-      """
-        |CREATE TABLE table_one_pass (ID int, date Timestamp, country String,
-        |name String, phonetype String, serialname String, salary int)
-        |STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-
-    sql(
-      s"""
-        |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass
-        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true')
-      """.stripMargin)
-  }
-
-  test("test data loading use one pass") {
-    checkAnswer(
-      sql("select * from table_one_pass"),
-      sql("select * from table_two_pass")
-    )
-  }
-
-  test("test data loading use one pass when offer column dictionary file") {
-    sql(
-      """
-        |CREATE TABLE table_one_pass_2 (ID int, date Timestamp, country String,
-        |name String, phonetype String, serialname String, salary int)
-        |STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""
-        |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass_2
-        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true', 'COLUMNDICT'=
-        |'country:$resourcesPath/columndictionary/country.csv, name:$resourcesPath/columndictionary/name.csv')
-      """.stripMargin)
-
-    checkAnswer(
-      sql("select * from table_one_pass_2"),
-      sql("select * from table_two_pass")
-    )
-  }
-
-  test("test data loading use one pass when do incremental load") {
-    sql(
-      s"""
-        |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_two_pass
-        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false')
-      """.stripMargin)
-    sql(
-      s"""
-        |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_one_pass
-        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true')
-      """.stripMargin)
-
-    checkAnswer(
-      sql("select * from table_one_pass"),
-      sql("select * from table_two_pass")
-    )
-  }
-
-  test("test data loading with dctionary exclude") {
-    sql("DROP TABLE IF EXISTS dict_exclude")
-    sql(
-      """
-        |CREATE TABLE dict_exclude (ID int, date Timestamp, country String,
-        |name String, phonetype String, serialname String, salary int)
-        |STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='country,name,serialname,phonetype')
-      """.stripMargin)
-    sql(
-      s"""
-         |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE dict_exclude
-         |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='FALSE')
-      """.stripMargin)
-    checkAnswer(sql("select name from dict_exclude limit 1"),Row("aaa1"))
-  }
-
-  override def afterAll {
-    sql("DROP TABLE IF EXISTS table_two_pass")
-    sql("DROP TABLE IF EXISTS table_one_pass")
-    sql("DROP TABLE IF EXISTS table_one_pass_2")
-    sql("DROP TABLE IF EXISTS dict_exclude")
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala
index 9b6780d..93ceeb9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala
@@ -278,7 +278,7 @@
 
     sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
         "serialname String, salary Int) STORED BY 'carbondata' " +
-        "tblproperties('sort_columns'='ID,name','dictionary_include'='ID')")
+        "tblproperties('sort_columns'='ID,name')")
     // ID is sort column and dictionary column. Since the actual order and literal order of
     // this column are not necessarily the same, this will not cause error but will cause data skewed.
     sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
@@ -297,7 +297,7 @@
 
     sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
         "serialname String, salary Int) STORED BY 'carbondata' " +
-        "tblproperties('sort_columns'='name,ID','dictionary_include'='name')")
+        "tblproperties('sort_columns'='name,ID')")
     // 'name' is sort column and dictionary column, but value for 'name' in bounds does not exists
     // in dictionary. It will not cause error but will cause data skewed.
     sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
index 0b0a321..f4c7d67 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
@@ -143,41 +143,6 @@
 
   }
 
-
-  test("no inverted index with measure as dictionary_include") {
-    sql("drop table if exists index2")
-
-    sql(
-      """
-        CREATE TABLE IF NOT EXISTS index2
-        (ID Int, date Timestamp, country String,
-        name String, phonetype String, serialname String, salary Int)
-        STORED BY 'org.apache.carbondata.format'
-        TBLPROPERTIES('DICTIONARY_INCLUDE'='ID','NO_INVERTED_INDEX'='ID')
-      """)
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-
-    sql(
-      s"""
-           LOAD DATA LOCAL INPATH '$testData2' into table index2
-           """)
-
-    checkAnswer(
-      sql(
-        """
-           SELECT country, count(salary) AS amount
-           FROM index2
-           WHERE country IN ('china','france')
-           GROUP BY country
-        """),
-      Seq(Row("china", 96), Row("france", 1))
-    )
-
-  }
-
-
   test("no inverted index with measure as sort_column") {
     sql("drop table if exists index2")
     sql(
@@ -208,14 +173,14 @@
 
   }
 
-  test("no inverted index with Dictionary_EXCLUDE and NO_INVERTED_INDEX") {
+  test("no inverted index with NO_INVERTED_INDEX") {
     sql("drop table if exists index1")
     sql(
       """
            CREATE TABLE IF NOT EXISTS index1
            (id Int, name String, city String)
            STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_EXCLUDE'='city','NO_INVERTED_INDEX'='city')
+           TBLPROPERTIES('NO_INVERTED_INDEX'='city')
       """)
     sql(
       s"""
@@ -234,7 +199,7 @@
            CREATE TABLE IF NOT EXISTS carbonNoInvertedIndexTable
            (id Int, name String, city String)
            STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('NO_INVERTED_INDEX'='name,city', 'DICTIONARY_EXCLUDE'='city')
+           TBLPROPERTIES('NO_INVERTED_INDEX'='name,city')
         """)
     sql(s"""
            LOAD DATA LOCAL INPATH '$testData1' into table carbonNoInvertedIndexTable
@@ -272,7 +237,7 @@
            CREATE TABLE IF NOT EXISTS indexFormat
            (id Int, name String, city String)
            STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_EXCLUDE'='city','NO_INVERTED_INDEX'='city')
+           TBLPROPERTIES('NO_INVERTED_INDEX'='city')
       """)
     sql(
       s"""
@@ -292,40 +257,12 @@
   }
 
   test("filter query on dictionary and no inverted index column where all values are null"){
-    sql("""create table testNull (c1 string,c2 int,c3 string,c5 string) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='C2','NO_INVERTED_INDEX'='C2')""")
+    sql("""create table testNull (c1 string,c2 int,c3 string,c5 string) STORED BY 'carbondata' TBLPROPERTIES('NO_INVERTED_INDEX'='C2')""")
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table testNull OPTIONS('delimiter'=';','fileheader'='c1,c2,c3,c5')""")
     sql("""select c2 from testNull where c2 is null""").show()
     checkAnswer(sql("""select c2 from testNull where c2 is null"""), Seq(Row(null), Row(null), Row(null), Row(null), Row(null), Row(null)))
   }
 
-  test("inverted index with Dictionary_INCLUDE and INVERTED_INDEX") {
-    sql("drop table if exists index1")
-    sql(
-      """
-           CREATE TABLE IF NOT EXISTS index1
-           (id Int, name String, city String)
-           STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_INCLUDE'='id','INVERTED_INDEX'='city,name', 'SORT_COLUMNS'='city,name')
-      """)
-    sql(
-      s"""
-           LOAD DATA LOCAL INPATH '$testData1' into table index1
-           """)
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "index1")
-    assert(carbonTable.getColumnByName("city").getColumnSchema.getEncodingList
-      .contains(Encoding.INVERTED_INDEX))
-    assert(carbonTable.getColumnByName("name").getColumnSchema.getEncodingList
-      .contains(Encoding.INVERTED_INDEX))
-    assert(!carbonTable.getColumnByName("id").getColumnSchema.getEncodingList
-      .contains(Encoding.INVERTED_INDEX))
-    checkAnswer(
-      sql(
-        """
-           SELECT * FROM index1 WHERE city = "Bangalore"
-        """),
-      Seq(Row(19.0, "Emily", "Bangalore")))
-  }
-
   test("inverted index with measure column in INVERTED_INDEX") {
     sql("drop table if exists index1")
     sql(
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/DoubleDataTypeTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/DoubleDataTypeTestCase.scala
index fe80cd3..1165d1e 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/DoubleDataTypeTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/DoubleDataTypeTestCase.scala
@@ -60,8 +60,6 @@
       .format("carbondata")
       .option("tableName", "doubleTypeCarbonTable")
       .option("tempCSV", "false")
-      .option("single_pass", "true")
-      .option("dictionary_exclude", "city")
       .option("table_blocksize", "32")
       .mode(SaveMode.Overwrite)
       .save()
@@ -79,7 +77,7 @@
   }
 
   test("duplicate values") {
-    sql("create table uniq_carbon(name string, double_column double) stored by 'carbondata' TBLPROPERTIES ('DICTIONARY_INCLUDE'='double_column')")
+    sql("create table uniq_carbon(name string, double_column double) stored by 'carbondata' ")
     sql(s"load data inpath '$resourcesPath/uniq.csv' into table uniq_carbon")
     sql("create table uniq_hive(name string, double_column double) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
     sql(s"load data local inpath '$resourcesPath/uniqwithoutheader.csv' into table uniq_hive")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala
index ca1a4ee..8f1134a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/primitiveTypes/TestAdaptiveEncodingForPrimitiveTypes.scala
@@ -98,9 +98,7 @@
       "CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB " +
       "timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 " +
       "decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-      "double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-      "BIGINT_COLUMN2,INTEGER_COLUMN1')")
+      "double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format'")
 
     sql(s"LOAD DATA INPATH '${ resourcesPath + "/data_with_all_types.csv" }' into table" +
         " uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='\"'," +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/aggquery/AverageQueryTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/aggquery/AverageQueryTestCase.scala
index 6561df2..c299763 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/aggquery/AverageQueryTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/aggquery/AverageQueryTestCase.scala
@@ -32,8 +32,7 @@
     sql("""
        CREATE TABLE carbonTable (ID int, date timeStamp, country string, count int,
        phonetype string, serialname string, salary double)
-       STORED BY 'org.apache.carbondata.format'
-        TBLPROPERTIES('DICTIONARY_INCLUDE'='ID')""")
+       STORED BY 'org.apache.carbondata.format'""")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/avgTest.csv' INTO table carbonTable""")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala
index 4c6b47a..9a0cd12 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase.scala
@@ -17,8 +17,7 @@
 
 package org.apache.carbondata.spark.testsuite.allqueries
 
-import org.apache.spark.sql.catalyst.plans.logical.Join
-import org.apache.spark.sql.{CarbonDictionaryCatalystDecoder, Row, SaveMode}
+import org.apache.spark.sql.{Row, SaveMode}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -34,7 +33,7 @@
   override def beforeAll {
     clean
 
-    sql("create table if not exists Carbon_automation_test (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Latest_MONTH,Latest_DAY,deviceInformationId')")
+    sql("create table if not exists Carbon_automation_test (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber int) STORED BY 'org.apache.carbondata.format'")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/100_olap.csv' INTO table Carbon_automation_test options('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'FILEHEADER'= 'imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""")
@@ -1156,53 +1155,10 @@
 
   }
 
-  test("TPCH query issue with not joining with decoded values") {
-
-    sql("drop table if exists SUPPLIER")
-    sql("drop table if exists PARTSUPP")
-    sql("drop table if exists CUSTOMER")
-    sql("drop table if exists NATION")
-    sql("drop table if exists REGION")
-    sql("drop table if exists PART")
-    sql("drop table if exists LINEITEM")
-    sql("drop table if exists ORDERS")
-    sql("create table if not exists SUPPLIER(S_COMMENT string,S_SUPPKEY string,S_NAME string, S_ADDRESS string, S_NATIONKEY string, S_PHONE string, S_ACCTBAL double) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_EXCLUDE'='S_COMMENT, S_SUPPKEY, S_NAME, S_ADDRESS, S_NATIONKEY, S_PHONE','table_blocksize'='300','SORT_COLUMNS'='')")
-    sql("create table if not exists PARTSUPP (  PS_PARTKEY int,  PS_SUPPKEY  string,  PS_AVAILQTY  int,  PS_SUPPLYCOST  double,  PS_COMMENT  string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_EXCLUDE'='PS_SUPPKEY,PS_COMMENT', 'table_blocksize'='300', 'no_inverted_index'='PS_SUPPKEY, PS_COMMENT','SORT_COLUMNS'='')")
-    sql("create table if not exists CUSTOMER(  C_MKTSEGMENT string,  C_NATIONKEY string,  C_CUSTKEY string,  C_NAME string,  C_ADDRESS string,  C_PHONE string,  C_ACCTBAL double,  C_COMMENT string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_INCLUDE'='C_MKTSEGMENT,C_NATIONKEY','DICTIONARY_EXCLUDE'='C_CUSTKEY,C_NAME,C_ADDRESS,C_PHONE,C_COMMENT', 'table_blocksize'='300', 'no_inverted_index'='C_CUSTKEY,C_NAME,C_ADDRESS,C_PHONE,C_COMMENT','SORT_COLUMNS'='C_MKTSEGMENT')")
-    sql("create table if not exists NATION (  N_NAME string,  N_NATIONKEY string,  N_REGIONKEY string,  N_COMMENT  string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_INCLUDE'='N_REGIONKEY','DICTIONARY_EXCLUDE'='N_COMMENT', 'table_blocksize'='300','no_inverted_index'='N_COMMENT','SORT_COLUMNS'='N_NAME')")
-    sql("create table if not exists REGION(  R_NAME string,  R_REGIONKEY string,  R_COMMENT string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_INCLUDE'='R_NAME,R_REGIONKEY','DICTIONARY_EXCLUDE'='R_COMMENT', 'table_blocksize'='300','no_inverted_index'='R_COMMENT','SORT_COLUMNS'='R_NAME')")
-    sql("create table if not exists PART(  P_BRAND string,  P_SIZE int,  P_CONTAINER string,  P_TYPE string,  P_PARTKEY INT ,  P_NAME string,  P_MFGR string,  P_RETAILPRICE double,  P_COMMENT string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_INCLUDE'='P_BRAND,P_SIZE,P_CONTAINER,P_MFGR','DICTIONARY_EXCLUDE'='P_NAME, P_COMMENT', 'table_blocksize'='300','no_inverted_index'='P_NAME,P_COMMENT,P_MFGR','SORT_COLUMNS'='P_SIZE,P_TYPE,P_NAME,P_BRAND,P_CONTAINER')")
-    sql("create table if not exists LINEITEM(  L_SHIPDATE date,  L_SHIPMODE string,  L_SHIPINSTRUCT string,  L_RETURNFLAG string,  L_RECEIPTDATE date,  L_ORDERKEY INT ,  L_PARTKEY INT ,  L_SUPPKEY   string,  L_LINENUMBER int,  L_QUANTITY double,  L_EXTENDEDPRICE double,  L_DISCOUNT double,  L_TAX double,  L_LINESTATUS string,  L_COMMITDATE date,  L_COMMENT  string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_INCLUDE'='L_SHIPDATE,L_SHIPMODE,L_SHIPINSTRUCT,L_RECEIPTDATE,L_COMMITDATE,L_RETURNFLAG,L_LINESTATUS','DICTIONARY_EXCLUDE'='L_SUPPKEY, L_COMMENT', 'table_blocksize'='300', 'no_inverted_index'='L_SUPPKEY,L_COMMENT','SORT_COLUMNS'='L_SHIPDATE,L_RETURNFLAG,L_SHIPMODE,L_RECEIPTDATE,L_SHIPINSTRUCT')")
-    sql("create table if not exists ORDERS(  O_ORDERDATE date,  O_ORDERPRIORITY string,  O_ORDERSTATUS string,  O_ORDERKEY int,  O_CUSTKEY string,  O_TOTALPRICE double,  O_CLERK string,  O_SHIPPRIORITY int,  O_COMMENT string) STORED BY 'org.apache.carbondata.format'TBLPROPERTIES ('DICTIONARY_INCLUDE'='O_ORDERDATE,O_ORDERSTATUS','DICTIONARY_EXCLUDE'='O_ORDERPRIORITY, O_CUSTKEY, O_CLERK, O_COMMENT', 'table_blocksize'='300','no_inverted_index'='O_ORDERPRIORITY, O_CUSTKEY, O_CLERK, O_COMMENT', 'SORT_COLUMNS'='O_ORDERDATE')")
-    val df = sql(
-      "select s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment from " +
-      "part, supplier, partsupp, nation, region where p_partkey = ps_partkey and s_suppkey = " +
-      "ps_suppkey and p_size = 15 and p_type like '%BRASS' and s_nationkey = n_nationkey and " +
-      "n_regionkey = r_regionkey and r_name = 'EUROPE' and ps_supplycost = ( select min" +
-      "(ps_supplycost) from partsupp, supplier,nation, region where p_partkey = ps_partkey and " +
-      "s_suppkey = ps_suppkey and s_nationkey = n_nationkey and n_regionkey = r_regionkey and " +
-      "r_name = 'EUROPE' ) order by s_acctbal desc, n_name, s_name, p_partkey limit 100")
-
-    val decoders = df.queryExecution.optimizedPlan.collect {
-      case p: CarbonDictionaryCatalystDecoder => p
-    }
-
-    assertResult(5)(decoders.length)
-
-    sql("drop table if exists SUPPLIER")
-    sql("drop table if exists PARTSUPP")
-    sql("drop table if exists CUSTOMER")
-    sql("drop table if exists NATION")
-    sql("drop table if exists REGION")
-    sql("drop table if exists PART")
-    sql("drop table if exists LINEITEM")
-    sql("drop table if exists ORDERS")
-  }
-
   test("test self join query fail") {
     sql("DROP TABLE IF EXISTS uniqdata_INCLUDEDICTIONARY")
 
-    sql("CREATE TABLE uniqdata_INCLUDEDICTIONARY (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql("CREATE TABLE uniqdata_INCLUDEDICTIONARY (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA INPATH '${resourcesPath + "/data_with_all_types.csv"}' into table" +
               " uniqdata_INCLUDEDICTIONARY OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='\"'," +
               "'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION," +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
index 2d5f05a..ce5114d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
@@ -36,10 +36,10 @@
     sql("drop table if exists carbon_datatype_AdaptiveFloatingCodec_double_int")
     sql("drop table if exists carbon_datatype_AdaptiveFloatingCodec_double_long")
     sql("DROP TABLE IF EXISTS oscon_carbon_old1")
-    sql("""create table oscon_carbon_old (CUST_PRFRD_FLG String,PROD_BRAND_NAME String,PROD_COLOR String,CUST_LAST_RVW_DATE String,CUST_COUNTRY String,CUST_CITY String,PRODUCT_NAME String,CUST_JOB_TITLE String,CUST_STATE String,CUST_BUY_POTENTIAL String,PRODUCT_MODEL String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,PROD_MODEL_ID String,CUST_ID String,PROD_UNQ_MDL_ID String,CUST_NICK_NAME String,CUST_LOGIN String,CUST_EMAIL_ADDR String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_BAR_CODE String,TRACKING_NO String,STR_ORDER_NO String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' tblproperties('DICTIONARY_EXCLUDE'='CUST_ID,CUST_NICK_NAME,CUST_FIRST_NAME,CUST_LAST_NAME,CUST_LOGIN,CUST_EMAIL_ADDR,PROD_UNQ_MDL_ID,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,DEVICE_NAME,PROD_BAR_CODE,ITM_ID,ITM_NAME,ITM_BRAND_ID,ITM_BRAND,BOM,PACKING_LIST_NO,TRACKING_NO,ACTIVE_FIRMWARE_VER,LATEST_FIRMWARE_VER,LATEST_EMUI_VERSION,LATEST_NETWORK,STR_ORDER_NO','table_blocksize'='256')""")
+    sql("""create table oscon_carbon_old (CUST_PRFRD_FLG String,PROD_BRAND_NAME String,PROD_COLOR String,CUST_LAST_RVW_DATE String,CUST_COUNTRY String,CUST_CITY String,PRODUCT_NAME String,CUST_JOB_TITLE String,CUST_STATE String,CUST_BUY_POTENTIAL String,PRODUCT_MODEL String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,PROD_MODEL_ID String,CUST_ID String,PROD_UNQ_MDL_ID String,CUST_NICK_NAME String,CUST_LOGIN String,CUST_EMAIL_ADDR String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_BAR_CODE String,TRACKING_NO String,STR_ORDER_NO String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' tblproperties ('table_blocksize'='256')""")
     sql(s"""load data LOCAL inpath '$resourcesPath/oscon_10.csv' into table oscon_carbon_old options('DELIMITER'=',', 'QUOTECHAR'='\"','FILEHEADER'='ACTIVE_AREA_ID, ACTIVE_CHECK_DY, ACTIVE_CHECK_HOUR, ACTIVE_CHECK_MM, ACTIVE_CHECK_TIME, ACTIVE_CHECK_YR, ACTIVE_CITY, ACTIVE_COUNTRY, ACTIVE_DISTRICT, ACTIVE_EMUI_VERSION, ACTIVE_FIRMWARE_VER, ACTIVE_NETWORK, ACTIVE_OS_VERSION, ACTIVE_PROVINCE, BOM, CHECK_DATE, CHECK_DY, CHECK_HOUR, CHECK_MM, CHECK_YR, CUST_ADDRESS_ID, CUST_AGE, CUST_BIRTH_COUNTRY, CUST_BIRTH_DY, CUST_BIRTH_MM, CUST_BIRTH_YR, CUST_BUY_POTENTIAL, CUST_CITY, CUST_STATE, CUST_COUNTRY, CUST_COUNTY, CUST_EMAIL_ADDR, CUST_LAST_RVW_DATE, CUST_FIRST_NAME, CUST_ID, CUST_JOB_TITLE, CUST_LAST_NAME, CUST_LOGIN, CUST_NICK_NAME, CUST_PRFRD_FLG, CUST_SEX, CUST_STREET_NAME, CUST_STREET_NO, CUST_SUITE_NO, CUST_ZIP, DELIVERY_CITY, DELIVERY_STATE, DELIVERY_COUNTRY, DELIVERY_DISTRICT, DELIVERY_PROVINCE, DEVICE_NAME, INSIDE_NAME, ITM_BRAND, ITM_BRAND_ID, ITM_CATEGORY, ITM_CATEGORY_ID, ITM_CLASS, ITM_CLASS_ID, ITM_COLOR, ITM_CONTAINER, ITM_FORMULATION, ITM_MANAGER_ID, ITM_MANUFACT, ITM_MANUFACT_ID, ITM_ID, ITM_NAME, ITM_REC_END_DATE, ITM_REC_START_DATE, LATEST_AREAID, LATEST_CHECK_DY, LATEST_CHECK_HOUR, LATEST_CHECK_MM, LATEST_CHECK_TIME, LATEST_CHECK_YR, LATEST_CITY, LATEST_COUNTRY, LATEST_DISTRICT, LATEST_EMUI_VERSION, LATEST_FIRMWARE_VER, LATEST_NETWORK, LATEST_OS_VERSION, LATEST_PROVINCE, OL_ORDER_DATE, OL_ORDER_NO, OL_RET_ORDER_NO, OL_RET_DATE, OL_SITE, OL_SITE_DESC, PACKING_DATE, PACKING_DY, PACKING_HOUR, PACKING_LIST_NO, PACKING_MM, PACKING_YR, PRMTION_ID, PRMTION_NAME, PRM_CHANNEL_CAT, PRM_CHANNEL_DEMO, PRM_CHANNEL_DETAILS, PRM_CHANNEL_DMAIL, PRM_CHANNEL_EMAIL, PRM_CHANNEL_EVENT, PRM_CHANNEL_PRESS, PRM_CHANNEL_RADIO, PRM_CHANNEL_TV, PRM_DSCNT_ACTIVE, PRM_END_DATE, PRM_PURPOSE, PRM_START_DATE, PRODUCT_ID, PROD_BAR_CODE, PROD_BRAND_NAME, PRODUCT_NAME, PRODUCT_MODEL, PROD_MODEL_ID, PROD_COLOR, PROD_SHELL_COLOR, PROD_CPU_CLOCK, PROD_IMAGE, PROD_LIVE, PROD_LOC, PROD_LONG_DESC, PROD_RAM, PROD_ROM, PROD_SERIES, PROD_SHORT_DESC, PROD_THUMB, PROD_UNQ_DEVICE_ADDR, PROD_UNQ_MDL_ID, PROD_UPDATE_DATE, PROD_UQ_UUID, SHP_CARRIER, SHP_CODE, SHP_CONTRACT, SHP_MODE_ID, SHP_MODE, STR_ORDER_DATE, STR_ORDER_NO, TRACKING_NO, WH_CITY, WH_COUNTRY, WH_COUNTY, WH_ID, WH_NAME, WH_STATE, WH_STREET_NAME, WH_STREET_NO, WH_STREET_TYPE, WH_SUITE_NO, WH_ZIP, CUST_DEP_COUNT, CUST_VEHICLE_COUNT, CUST_ADDRESS_CNT, CUST_CRNT_CDEMO_CNT, CUST_CRNT_HDEMO_CNT, CUST_CRNT_ADDR_DM, CUST_FIRST_SHIPTO_CNT, CUST_FIRST_SALES_CNT, CUST_GMT_OFFSET, CUST_DEMO_CNT, CUST_INCOME, PROD_UNLIMITED, PROD_OFF_PRICE, PROD_UNITS, TOTAL_PRD_COST, TOTAL_PRD_DISC, PROD_WEIGHT, REG_UNIT_PRICE, EXTENDED_AMT, UNIT_PRICE_DSCNT_PCT, DSCNT_AMT, PROD_STD_CST, TOTAL_TX_AMT, FREIGHT_CHRG, WAITING_PERIOD, DELIVERY_PERIOD, ITM_CRNT_PRICE, ITM_UNITS, ITM_WSLE_CST, ITM_SIZE, PRM_CST, PRM_RESPONSE_TARGET, PRM_ITM_DM, SHP_MODE_CNT, WH_GMT_OFFSET, WH_SQ_FT, STR_ORD_QTY, STR_WSLE_CST, STR_LIST_PRICE, STR_SALES_PRICE, STR_EXT_DSCNT_AMT, STR_EXT_SALES_PRICE, STR_EXT_WSLE_CST, STR_EXT_LIST_PRICE, STR_EXT_TX, STR_COUPON_AMT, STR_NET_PAID, STR_NET_PAID_INC_TX, STR_NET_PRFT, STR_SOLD_YR_CNT, STR_SOLD_MM_CNT, STR_SOLD_ITM_CNT, STR_TOTAL_CUST_CNT, STR_AREA_CNT, STR_DEMO_CNT, STR_OFFER_CNT, STR_PRM_CNT, STR_TICKET_CNT, STR_NET_PRFT_DM_A, STR_NET_PRFT_DM_B, STR_NET_PRFT_DM_C, STR_NET_PRFT_DM_D, STR_NET_PRFT_DM_E, STR_RET_STR_ID, STR_RET_REASON_CNT, STR_RET_TICKET_NO, STR_RTRN_QTY, STR_RTRN_AMT, STR_RTRN_TX, STR_RTRN_AMT_INC_TX, STR_RET_FEE, STR_RTRN_SHIP_CST, STR_RFNDD_CSH, STR_REVERSED_CHRG, STR_STR_CREDIT, STR_RET_NET_LOSS, STR_RTRNED_YR_CNT, STR_RTRN_MM_CNT, STR_RET_ITM_CNT, STR_RET_CUST_CNT, STR_RET_AREA_CNT, STR_RET_OFFER_CNT, STR_RET_PRM_CNT, STR_RET_NET_LOSS_DM_A, STR_RET_NET_LOSS_DM_B, STR_RET_NET_LOSS_DM_C, STR_RET_NET_LOSS_DM_D, OL_ORD_QTY, OL_WSLE_CST, OL_LIST_PRICE, OL_SALES_PRICE, OL_EXT_DSCNT_AMT, OL_EXT_SALES_PRICE, OL_EXT_WSLE_CST, OL_EXT_LIST_PRICE, OL_EXT_TX, OL_COUPON_AMT, OL_EXT_SHIP_CST, OL_NET_PAID, OL_NET_PAID_INC_TX, OL_NET_PAID_INC_SHIP, OL_NET_PAID_INC_SHIP_TX, OL_NET_PRFT, OL_SOLD_YR_CNT, OL_SOLD_MM_CNT, OL_SHIP_DATE_CNT, OL_ITM_CNT, OL_BILL_CUST_CNT, OL_BILL_AREA_CNT, OL_BILL_DEMO_CNT, OL_BILL_OFFER_CNT, OL_SHIP_CUST_CNT, OL_SHIP_AREA_CNT, OL_SHIP_DEMO_CNT, OL_SHIP_OFFER_CNT, OL_WEB_PAGE_CNT, OL_WEB_SITE_CNT, OL_SHIP_MODE_CNT, OL_WH_CNT, OL_PRM_CNT, OL_NET_PRFT_DM_A, OL_NET_PRFT_DM_B, OL_NET_PRFT_DM_C, OL_NET_PRFT_DM_D, OL_RET_RTRN_QTY, OL_RTRN_AMT, OL_RTRN_TX, OL_RTRN_AMT_INC_TX, OL_RET_FEE, OL_RTRN_SHIP_CST, OL_RFNDD_CSH, OL_REVERSED_CHRG, OL_ACCOUNT_CREDIT, OL_RTRNED_YR_CNT, OL_RTRNED_MM_CNT, OL_RTRITM_CNT, OL_RFNDD_CUST_CNT, OL_RFNDD_AREA_CNT, OL_RFNDD_DEMO_CNT, OL_RFNDD_OFFER_CNT, OL_RTRNING_CUST_CNT, OL_RTRNING_AREA_CNT, OL_RTRNING_DEMO_CNT, OL_RTRNING_OFFER_CNT, OL_RTRWEB_PAGE_CNT, OL_REASON_CNT, OL_NET_LOSS, OL_NET_LOSS_DM_A, OL_NET_LOSS_DM_B, OL_NET_LOSS_DM_C','BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='FALSE')""")
     sql(
-      """create table oscon_carbon_old1 (CUST_PRFRD_FLG String,PROD_BRAND_NAME String,PROD_COLOR String,CUST_LAST_RVW_DATE String,CUST_COUNTRY String,CUST_CITY String,PRODUCT_NAME String,CUST_JOB_TITLE String,CUST_STATE String,CUST_BUY_POTENTIAL String,PRODUCT_MODEL String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,PROD_MODEL_ID String,CUST_ID String,PROD_UNQ_MDL_ID String,CUST_NICK_NAME String,CUST_LOGIN String,CUST_EMAIL_ADDR String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_BAR_CODE String,TRACKING_NO String,STR_ORDER_NO String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' tblproperties('DICTIONARY_INCLUDE'='CUST_DEP_COUNT','DICTIONARY_EXCLUDE'='CUST_ID,CUST_NICK_NAME,CUST_FIRST_NAME,CUST_LAST_NAME,CUST_LOGIN,CUST_EMAIL_ADDR,PROD_UNQ_MDL_ID,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,DEVICE_NAME,PROD_BAR_CODE,ITM_ID,ITM_NAME,ITM_BRAND_ID,ITM_BRAND,BOM,PACKING_LIST_NO,TRACKING_NO,ACTIVE_FIRMWARE_VER,LATEST_FIRMWARE_VER,LATEST_EMUI_VERSION,LATEST_NETWORK,STR_ORDER_NO','table_blocksize'='256')""".stripMargin)
+      """create table oscon_carbon_old1 (CUST_PRFRD_FLG String,PROD_BRAND_NAME String,PROD_COLOR String,CUST_LAST_RVW_DATE String,CUST_COUNTRY String,CUST_CITY String,PRODUCT_NAME String,CUST_JOB_TITLE String,CUST_STATE String,CUST_BUY_POTENTIAL String,PRODUCT_MODEL String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,PROD_MODEL_ID String,CUST_ID String,PROD_UNQ_MDL_ID String,CUST_NICK_NAME String,CUST_LOGIN String,CUST_EMAIL_ADDR String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_BAR_CODE String,TRACKING_NO String,STR_ORDER_NO String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('table_blocksize'='256')""".stripMargin)
     sql(s"""load data LOCAL inpath '$resourcesPath/oscon_10.csv' into table oscon_carbon_old1 options('DELIMITER'=',', 'QUOTECHAR'='\"','FILEHEADER'='ACTIVE_AREA_ID, ACTIVE_CHECK_DY, ACTIVE_CHECK_HOUR, ACTIVE_CHECK_MM, ACTIVE_CHECK_TIME, ACTIVE_CHECK_YR, ACTIVE_CITY, ACTIVE_COUNTRY, ACTIVE_DISTRICT, ACTIVE_EMUI_VERSION, ACTIVE_FIRMWARE_VER, ACTIVE_NETWORK, ACTIVE_OS_VERSION, ACTIVE_PROVINCE, BOM, CHECK_DATE, CHECK_DY, CHECK_HOUR, CHECK_MM, CHECK_YR, CUST_ADDRESS_ID, CUST_AGE, CUST_BIRTH_COUNTRY, CUST_BIRTH_DY, CUST_BIRTH_MM, CUST_BIRTH_YR, CUST_BUY_POTENTIAL, CUST_CITY, CUST_STATE, CUST_COUNTRY, CUST_COUNTY, CUST_EMAIL_ADDR, CUST_LAST_RVW_DATE, CUST_FIRST_NAME, CUST_ID, CUST_JOB_TITLE, CUST_LAST_NAME, CUST_LOGIN, CUST_NICK_NAME, CUST_PRFRD_FLG, CUST_SEX, CUST_STREET_NAME, CUST_STREET_NO, CUST_SUITE_NO, CUST_ZIP, DELIVERY_CITY, DELIVERY_STATE, DELIVERY_COUNTRY, DELIVERY_DISTRICT, DELIVERY_PROVINCE, DEVICE_NAME, INSIDE_NAME, ITM_BRAND, ITM_BRAND_ID, ITM_CATEGORY, ITM_CATEGORY_ID, ITM_CLASS, ITM_CLASS_ID, ITM_COLOR, ITM_CONTAINER, ITM_FORMULATION, ITM_MANAGER_ID, ITM_MANUFACT, ITM_MANUFACT_ID, ITM_ID, ITM_NAME, ITM_REC_END_DATE, ITM_REC_START_DATE, LATEST_AREAID, LATEST_CHECK_DY, LATEST_CHECK_HOUR, LATEST_CHECK_MM, LATEST_CHECK_TIME, LATEST_CHECK_YR, LATEST_CITY, LATEST_COUNTRY, LATEST_DISTRICT, LATEST_EMUI_VERSION, LATEST_FIRMWARE_VER, LATEST_NETWORK, LATEST_OS_VERSION, LATEST_PROVINCE, OL_ORDER_DATE, OL_ORDER_NO, OL_RET_ORDER_NO, OL_RET_DATE, OL_SITE, OL_SITE_DESC, PACKING_DATE, PACKING_DY, PACKING_HOUR, PACKING_LIST_NO, PACKING_MM, PACKING_YR, PRMTION_ID, PRMTION_NAME, PRM_CHANNEL_CAT, PRM_CHANNEL_DEMO, PRM_CHANNEL_DETAILS, PRM_CHANNEL_DMAIL, PRM_CHANNEL_EMAIL, PRM_CHANNEL_EVENT, PRM_CHANNEL_PRESS, PRM_CHANNEL_RADIO, PRM_CHANNEL_TV, PRM_DSCNT_ACTIVE, PRM_END_DATE, PRM_PURPOSE, PRM_START_DATE, PRODUCT_ID, PROD_BAR_CODE, PROD_BRAND_NAME, PRODUCT_NAME, PRODUCT_MODEL, PROD_MODEL_ID, PROD_COLOR, PROD_SHELL_COLOR, PROD_CPU_CLOCK, PROD_IMAGE, PROD_LIVE, PROD_LOC, PROD_LONG_DESC, PROD_RAM, PROD_ROM, PROD_SERIES, PROD_SHORT_DESC, PROD_THUMB, PROD_UNQ_DEVICE_ADDR, PROD_UNQ_MDL_ID, PROD_UPDATE_DATE, PROD_UQ_UUID, SHP_CARRIER, SHP_CODE, SHP_CONTRACT, SHP_MODE_ID, SHP_MODE, STR_ORDER_DATE, STR_ORDER_NO, TRACKING_NO, WH_CITY, WH_COUNTRY, WH_COUNTY, WH_ID, WH_NAME, WH_STATE, WH_STREET_NAME, WH_STREET_NO, WH_STREET_TYPE, WH_SUITE_NO, WH_ZIP, CUST_DEP_COUNT, CUST_VEHICLE_COUNT, CUST_ADDRESS_CNT, CUST_CRNT_CDEMO_CNT, CUST_CRNT_HDEMO_CNT, CUST_CRNT_ADDR_DM, CUST_FIRST_SHIPTO_CNT, CUST_FIRST_SALES_CNT, CUST_GMT_OFFSET, CUST_DEMO_CNT, CUST_INCOME, PROD_UNLIMITED, PROD_OFF_PRICE, PROD_UNITS, TOTAL_PRD_COST, TOTAL_PRD_DISC, PROD_WEIGHT, REG_UNIT_PRICE, EXTENDED_AMT, UNIT_PRICE_DSCNT_PCT, DSCNT_AMT, PROD_STD_CST, TOTAL_TX_AMT, FREIGHT_CHRG, WAITING_PERIOD, DELIVERY_PERIOD, ITM_CRNT_PRICE, ITM_UNITS, ITM_WSLE_CST, ITM_SIZE, PRM_CST, PRM_RESPONSE_TARGET, PRM_ITM_DM, SHP_MODE_CNT, WH_GMT_OFFSET, WH_SQ_FT, STR_ORD_QTY, STR_WSLE_CST, STR_LIST_PRICE, STR_SALES_PRICE, STR_EXT_DSCNT_AMT, STR_EXT_SALES_PRICE, STR_EXT_WSLE_CST, STR_EXT_LIST_PRICE, STR_EXT_TX, STR_COUPON_AMT, STR_NET_PAID, STR_NET_PAID_INC_TX, STR_NET_PRFT, STR_SOLD_YR_CNT, STR_SOLD_MM_CNT, STR_SOLD_ITM_CNT, STR_TOTAL_CUST_CNT, STR_AREA_CNT, STR_DEMO_CNT, STR_OFFER_CNT, STR_PRM_CNT, STR_TICKET_CNT, STR_NET_PRFT_DM_A, STR_NET_PRFT_DM_B, STR_NET_PRFT_DM_C, STR_NET_PRFT_DM_D, STR_NET_PRFT_DM_E, STR_RET_STR_ID, STR_RET_REASON_CNT, STR_RET_TICKET_NO, STR_RTRN_QTY, STR_RTRN_AMT, STR_RTRN_TX, STR_RTRN_AMT_INC_TX, STR_RET_FEE, STR_RTRN_SHIP_CST, STR_RFNDD_CSH, STR_REVERSED_CHRG, STR_STR_CREDIT, STR_RET_NET_LOSS, STR_RTRNED_YR_CNT, STR_RTRN_MM_CNT, STR_RET_ITM_CNT, STR_RET_CUST_CNT, STR_RET_AREA_CNT, STR_RET_OFFER_CNT, STR_RET_PRM_CNT, STR_RET_NET_LOSS_DM_A, STR_RET_NET_LOSS_DM_B, STR_RET_NET_LOSS_DM_C, STR_RET_NET_LOSS_DM_D, OL_ORD_QTY, OL_WSLE_CST, OL_LIST_PRICE, OL_SALES_PRICE, OL_EXT_DSCNT_AMT, OL_EXT_SALES_PRICE, OL_EXT_WSLE_CST, OL_EXT_LIST_PRICE, OL_EXT_TX, OL_COUPON_AMT, OL_EXT_SHIP_CST, OL_NET_PAID, OL_NET_PAID_INC_TX, OL_NET_PAID_INC_SHIP, OL_NET_PAID_INC_SHIP_TX, OL_NET_PRFT, OL_SOLD_YR_CNT, OL_SOLD_MM_CNT, OL_SHIP_DATE_CNT, OL_ITM_CNT, OL_BILL_CUST_CNT, OL_BILL_AREA_CNT, OL_BILL_DEMO_CNT, OL_BILL_OFFER_CNT, OL_SHIP_CUST_CNT, OL_SHIP_AREA_CNT, OL_SHIP_DEMO_CNT, OL_SHIP_OFFER_CNT, OL_WEB_PAGE_CNT, OL_WEB_SITE_CNT, OL_SHIP_MODE_CNT, OL_WH_CNT, OL_PRM_CNT, OL_NET_PRFT_DM_A, OL_NET_PRFT_DM_B, OL_NET_PRFT_DM_C, OL_NET_PRFT_DM_D, OL_RET_RTRN_QTY, OL_RTRN_AMT, OL_RTRN_TX, OL_RTRN_AMT_INC_TX, OL_RET_FEE, OL_RTRN_SHIP_CST, OL_RFNDD_CSH, OL_REVERSED_CHRG, OL_ACCOUNT_CREDIT, OL_RTRNED_YR_CNT, OL_RTRNED_MM_CNT, OL_RTRITM_CNT, OL_RFNDD_CUST_CNT, OL_RFNDD_AREA_CNT, OL_RFNDD_DEMO_CNT, OL_RFNDD_OFFER_CNT, OL_RTRNING_CUST_CNT, OL_RTRNING_AREA_CNT, OL_RTRNING_DEMO_CNT, OL_RTRNING_OFFER_CNT, OL_RTRWEB_PAGE_CNT, OL_REASON_CNT, OL_NET_LOSS, OL_NET_LOSS_DM_A, OL_NET_LOSS_DM_B, OL_NET_LOSS_DM_C','BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='FALSE')""")
   }
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
index 8487b9e..fb1c801 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
@@ -117,7 +117,6 @@
          |    floatField float
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
 
     sql(
@@ -135,7 +134,6 @@
          |    floatField float
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
     sql(
       s"""
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
index 5f003fe..3c15eb8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
@@ -61,7 +61,7 @@
   }
 
   private def createAndLoadTable(cacheLevel: String): Unit = {
-    sql(s"CREATE table column_min_max_cache_test (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp, attendance int, utilization int,salary int) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='empno','column_meta_cache'='workgroupcategoryname,designation,salary,attendance', 'CACHE_LEVEL'= '$cacheLevel')")
+    sql(s"CREATE table column_min_max_cache_test (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp, attendance int, utilization int,salary int) STORED BY 'carbondata' TBLPROPERTIES('column_meta_cache'='workgroupcategoryname,designation,salary,attendance', 'CACHE_LEVEL'= '$cacheLevel')")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO " +
         "TABLE column_min_max_cache_test OPTIONS('DELIMITER'=',', " +
         "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE')")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithoutDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithoutDataLoad.scala
index 3e4f2e5..4e549b0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithoutDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithoutDataLoad.scala
@@ -31,7 +31,6 @@
     sql("""
         CREATE TABLE no_load(imei string, age int, productdate timestamp, gamePointId double)
         STORED BY 'org.apache.carbondata.format'
-        TBLPROPERTIES('DICTIONARY_INCLUDE'='gamePointId')
       """)
   }
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
index c38d198..324fae8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
@@ -63,16 +63,16 @@
     )
     createTable(
       "alter_sc_validate",
-      Map("dictionary_include"->"charField"),
+      Map.empty,
       true
     )
     createTable(
       "alter_sc_iud",
-      Map("dictionary_include"->"charField")
+      Map.empty
     )
     createTable(
       "alter_sc_iud_complex",
-      Map("dictionary_include"->"charField"),
+      Map.empty,
       true
     )
     createTable(
@@ -138,7 +138,7 @@
 
     createTable(
       "alter_sc_cli",
-      Map("dictionary_include"->"charField")
+      Map.empty
     )
   }
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
index 551b00b..a880c59 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
@@ -50,7 +50,7 @@
     sql("create table if not exists carbonBigDecimal_2 (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(30, 10)) STORED BY 'org.apache.carbondata.format'")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalBoundaryDataCarbon.csv' into table carbonBigDecimal_2")
 
-    sql("create table if not exists carbonBigDecimal_3 (ID Int, date Timestamp, country String,name String, phonetype String, serialname String, salary decimal(30, 2)) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='salary')")
+    sql("create table if not exists carbonBigDecimal_3 (ID Int, date Timestamp, country String,name String, phonetype String, serialname String, salary decimal(30, 2)) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalBoundaryDataCarbon.csv' into table carbonBigDecimal_3")
   }
 
@@ -135,7 +135,7 @@
 
   test("test big decimal for dictionary look up") {
     sql("drop table if exists decimalDictLookUp")
-    sql("create table if not exists decimalDictLookUp (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(27, 10)) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='salary')")
+    sql("create table if not exists decimalDictLookUp (ID Int, date Timestamp, country String, name String, phonetype String, serialname String, salary decimal(27, 10)) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/decimalBoundaryDataCarbon.csv' into table decimalDictLookUp")
 
     checkAnswer(sql("select sum(salary) from decimalDictLookUp"),
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
index 95676d0..0ea5cb9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
@@ -65,8 +65,7 @@
       }
       writer.close()
     } catch {
-      case ex: Exception => None
-      case _ => None
+      case _: Throwable => None
     }
   }
 
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
index 5e10a74..237ba22 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
@@ -234,7 +234,6 @@
          | mapField map<STRING,STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('DICTIONARY_INCLUDE'='mapField')
          | """
         .stripMargin)
     sql("insert into carbon values(map('vi','Nalla','sh','Singh','al','Gupta'))")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
index f0e7a1e..483d9f5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
@@ -138,7 +138,7 @@
     sql("create table rstest1 (c1 string,c2 int) STORED BY 'org.apache.carbondata.format'")
     sql("Alter table rstest1 drop columns(c2)")
     sql(
-      "Alter table rstest1 add columns(c4 string) TBLPROPERTIES('DICTIONARY_EXCLUDE'='c4', " +
+      "Alter table rstest1 add columns(c4 string) TBLPROPERTIES( " +
       "'DEFAULT.VALUE.c4'='def')")
     sql(s"""CREATE EXTERNAL TABLE rsext STORED BY 'carbondata' LOCATION '$storeLocation/rstest1'""")
     sql("insert into rsext select 'shahid', 1")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
index 8e4d8fa..24b4687 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
@@ -149,19 +149,13 @@
     sql("DROP TABLE IF EXISTS ctas_tblproperties_testt")
     sql(
       "create table ctas_tblproperties_testt stored by 'carbondata' TBLPROPERTIES" +
-        "('DICTIONARY_INCLUDE'='key', 'sort_scope'='global_sort') as select * from carbon_ctas_test")
+        "('sort_scope'='global_sort') as select * from carbon_ctas_test")
     checkAnswer(sql("select * from ctas_tblproperties_testt"), sql("select * from carbon_ctas_test"))
     val carbonTable = CarbonEnv.getInstance(Spark2TestQueryExecutor.spark).carbonMetaStore
       .lookupRelation(Option("default"), "ctas_tblproperties_testt")(Spark2TestQueryExecutor.spark)
       .asInstanceOf[CarbonRelation].carbonTable
     val metadataFolderPath: CarbonFile = FileFactory.getCarbonFile(carbonTable.getMetadataPath)
     assert(metadataFolderPath.exists())
-    val dictFiles: Array[CarbonFile] = metadataFolderPath.listFiles(new CarbonFileFilter {
-      override def accept(file: CarbonFile): Boolean = {
-        file.getName.contains(".dict") || file.getName.contains(".sortindex")
-      }
-    })
-    assert(dictFiles.length == 3)
   }
 
   test("test create table as select with column name as tupleid") {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
index 0815eab..8682b4c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
@@ -41,7 +41,6 @@
         |TBLPROPERTIES(
         |  'SORT_COLUMNS'='b',
         |  'SORT_SCOPE'='GLOBAL_SORT',
-        |  'DICTIONARY_INCLUDE'='b',
         |  'LOCAL_DICTIONARY_ENABLE'='false',
         |  'carbon.column.compress'='zstd',
         |  'CACHE_LEVEL'='blocklet',
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
index 0360e35..d8a8825 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
@@ -2693,8 +2693,7 @@
       val decoder = encodingFactory.createDecoder(encodings, encoderMetas, compressorName)
       val dictionaryPage = decoder
         .decode(local_dictionary.getDictionary_data, 0, local_dictionary.getDictionary_data.length)
-      val dictionaryMap = new
-          util.HashMap[DictionaryByteArrayWrapper, Integer]
+      val dictionaryMap = new util.HashMap[DictionaryByteArrayWrapper, Integer]
       val usedDictionaryValues = util.BitSet
         .valueOf(CompressorFactory.getInstance.getCompressor(compressorName)
           .unCompressByte(local_dictionary.getDictionary_values))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
index 4e5ebbb..06918a9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
@@ -115,9 +115,7 @@
         "String, doj String, " +

         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +

         "projectcode string, projectjoindate String, projectenddate String,attendance double," +

-        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +

-        "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +

-        "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')")

+        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' ")

       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' into table boundary_max_columns_test" +

           s" options('MAXCOLUMNS'='14')")

 

@@ -131,15 +129,13 @@
         "CREATE TABLE boundary_max_columns_test (empno string, empname String, designation String, doj String, " +

         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +

         "projectcode string, projectjoindate String, projectenddate String,attendance double," +

-        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +

-        "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +

-        "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')")

+        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' ")

       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' into table boundary_max_columns_test options('MAXCOLUMNS'='13')")

     }

   }

 

   test("test for duplicate column name in the Fileheader options in load command") {

-    sql("create table smart_500_DE (MSISDN string,IMSI string,IMEI string,INTERFACEID int,GROUPID int,GGSN_ID double,SGSN_ID double,SESSION_INDICATOR int,BEGIN_TIME double,BEGIN_TIME_MSEL int,END_TIME double,END_TIME_MSEL int,PROT_CATEGORY int,PROT_TYPE int,L7_CARRIER_PROT int,SUB_PROT_TYPE int,SID double,PROBEID double,ENCRYPT_VERSION int,ROAMING_TYPE int,ROAM_DIRECTION int,MS_IP string,SERVER_IP string,MS_PORT int,SERVER_DECIMAL Decimal,APN string,SGSN_SIG_IP string,GGSN_SIG_IP_BigInt_NEGATIVE bigint,SGSN_USER_IP string,GGSN_USER_IP string,MCC string,MNC string,RAT int,LAC string,RAC string,SAC string,CI string,BROWSER_TIMESTAMP timestamp,TCP_CONN_STATES int,TCP_STATES_BIGINTPOSITIVE int,TCP_WIN_SIZE int,TCP_MSS int,TCP_CONN_TIMES int,TCP_CONN_2_FAILED_TIMES int,TCP_CONN_3_FAILED_TIMES int,HOST string,STREAMING_URL string,GET_STREAMING_FAILED_CODE int,GET_STREAMING_FLAG int,GET_NUM int,GET_SUCCEED_NUM int,GET_RETRANS_NUM int,GET_TIMEOUT_NUM int,INTBUFFER_FST_FLAG int,INTBUFFER_FULL_FLAG int,STALL_NUM int,VIDEO_FRAME_RATE int,VIDEO_CODEC_ID string,VIDEO_WIDTH int,VIDEO_HEIGHT int,AUDIO_CODEC_ID string,MEDIA_FILE_TYPE int,PLAY_STATE int,STREAMING_FLAG int,TCP_STATUS_INDICATOR int,DISCONNECTION_FLAG int,FAILURE_CODE int,FLAG int,TAC string,ECI string,TCP_SYN_TIME_MSEL int,TCP_FST_SYN_DIRECTION int,RAN_NE_USER_IP string,HOMEMCC string,HOMEMNC string,CHARGE_FLAG int,PREPAID_FLAG int,USER_AGENT string,MS_WIN_STAT_TOTAL_NUM int,MS_WIN_STAT_SMALL_NUM int,MS_ACK_TO_1STGET_DELAY int,SERVER_ACK_TO_1STDATA_DELAY int,STREAMING_TYPE int,SOURCE_VIDEO_QUALITY int,TETHERING_FLAG int,CARRIER_ID double,LAYER1ID int,LAYER2ID int,LAYER3ID int,LAYER4ID int,LAYER5ID int,LAYER6ID int,CHARGING_RULE_BASE_NAME string,SP string,EXTENDED_URL string,SV string,FIRST_SAI_CGI_ECGI string,EXTENDED_URL_OTHER string,SIGNALING_USE_FLAG int,DNS_RETRANS_NUM int,DNS_FAIL_CODE int,FIRST_RAT int,MS_INDICATOR string,LAST_SAI_CGI_ECGI string,LAST_RAT int,FIRST_LONGITUDE double,FIRST_LATITUDE double,FIRST_ALTITUDE int,FIRST_RASTERLONGITUDE double,FIRST_RASTERLATITUDE double,FIRST_RASTERALTITUDE int,FIRST_FREQUENCYSPOT int,FIRST_CLUTTER int,FIRST_USERBEHAVIOR int,FIRST_SPEED int,FIRST_CREDIBILITY int,LAST_LONGITUDE double,LAST_LATITUDE double,LAST_ALTITUDE int,LAST_RASTERLONGITUDE double,LAST_RASTERLATITUDE double,LAST_RASTERALTITUDE int,LAST_FREQUENCYSPOT int,LAST_CLUTTER int,LAST_USERBEHAVIOR int,LAST_SPEED int,LAST_CREDIBILITY int,IMEI_CIPHERTEXT string,APP_ID int,DOMAIN_NAME string,STREAMING_CACHE_IP string,STOP_LONGER_THAN_MIN_THRESHOLD int,STOP_LONGER_THAN_MAX_THRESHOLD int,PLAY_END_STAT int,STOP_START_TIME1 double,STOP_END_TIME1 double,STOP_START_TIME2 double,STOP_END_TIME2 double,STOP_START_TIME3 double,STOP_END_TIME3 double,STOP_START_TIME4 double,STOP_END_TIME4 double,STOP_START_TIME5 double,STOP_END_TIME5 double,STOP_START_TIME6 double,STOP_END_TIME6 double,STOP_START_TIME7 double,STOP_END_TIME7 double,STOP_START_TIME8 double,STOP_END_TIME8 double,STOP_START_TIME9 double,STOP_END_TIME9 double,STOP_START_TIME10 double,STOP_END_TIME10 double,FAIL_CLASS double,RECORD_TYPE double,NODATA_COUNT double,VIDEO_NODATA_DURATION double,VIDEO_SMOOTH_DURATION double,VIDEO_SD_DURATION double,VIDEO_HD_DURATION double,VIDEO_UHD_DURATION double,VIDEO_FHD_DURATION double,FLUCTUATION double,START_DOWNLOAD_THROUGHPUT double,L7_UL_GOODPUT_FULL_MSS double,SESSIONKEY string,FIRST_UCELLID double,LAST_UCELLID double,UCELLID1 double,LONGITUDE1 double,LATITUDE1 double,UCELLID2 double,LONGITUDE2 double,LATITUDE2 double,UCELLID3 double,LONGITUDE3 double,LATITUDE3 double,UCELLID4 double,LONGITUDE4 double,LATITUDE4 double,UCELLID5 double,LONGITUDE5 double,LATITUDE5 double,UCELLID6 double,LONGITUDE6 double,LATITUDE6 double,UCELLID7 double,LONGITUDE7 double,LATITUDE7 double,UCELLID8 double,LONGITUDE8 double,LATITUDE8 double,UCELLID9 double,LONGITUDE9 double,LATITUDE9 double,UCELLID10 double,LONGITUDE10 double,LATITUDE10 double,INTBUFFER_FULL_DELAY double,STALL_DURATION double,STREAMING_DW_PACKETS double,STREAMING_DOWNLOAD_DELAY double,PLAY_DURATION double,STREAMING_QUALITY int,VIDEO_DATA_RATE double,AUDIO_DATA_RATE double,STREAMING_FILESIZE double,STREAMING_DURATIOIN double,TCP_SYN_TIME double,TCP_RTT_STEP1 double,CHARGE_ID double,UL_REVERSE_TO_DL_DELAY double,DL_REVERSE_TO_UL_DELAY double,DATATRANS_DW_GOODPUT double,DATATRANS_DW_TOTAL_DURATION double,SUM_FRAGMENT_INTERVAL double,TCP_FIN_TIMES double,TCP_RESET_TIMES double,URL_CLASSIFICATION double,STREAMING_LQ_DURATIOIN double,MAX_DNS_DELAY double,MAX_DNS2SYN double,MAX_LATANCY_OF_LINK_SETUP double,MAX_SYNACK2FIRSTACK double,MAX_SYNACK2LASTACK double,MAX_ACK2GET_DELAY double,MAX_FRAG_INTERVAL_PREDELAY double,SUM_FRAG_INTERVAL_PREDELAY double,SERVICE_DELAY_MSEC double,HOMEPROVINCE double,HOMECITY double,SERVICE_ID double,CHARGING_CLASS double,DATATRANS_UL_DURATION double,ASSOCIATED_ID double,PACKET_LOSS_NUM double,JITTER double,MS_DNS_DELAY_MSEL double,GET_STREAMING_DELAY double,TCP_UL_RETRANS_WITHOUTPL double,TCP_DW_RETRANS_WITHOUTPL double,GET_MAX_UL_SIZE double,GET_MIN_UL_SIZE double,GET_MAX_DL_SIZE double,GET_MIN_DL_SIZE double,L4_UL_THROUGHPUT double,L4_DW_THROUGHPUT double,L4_UL_GOODPUT double,L4_DW_GOODPUT double,NETWORK_UL_TRAFFIC double,NETWORK_DL_TRAFFIC double,L4_UL_PACKETS double,L4_DW_PACKETS double,TCP_RTT double,TCP_UL_OUTOFSEQU double,TCP_DW_OUTOFSEQU double,TCP_UL_RETRANS double,TCP_DW_RETRANS double,TCP_UL_RETRANS_WITHPL double,TCP_DW_RETRANS_WITHPL double,TCP_UL_PACKAGES_WITHPL double,TCP_DW_PACKAGES_WITHPL double,TCP_UL_PACKAGES_WITHOUTPL double,TCP_DW_PACKAGES_WITHOUTPL double,UPPERLAYER_IP_UL_PACKETS double,UPPERLAYER_IP_DL_PACKETS double,DOWNLAYER_IP_UL_PACKETS double,DOWNLAYER_IP_DL_PACKETS double,UPPERLAYER_IP_UL_FRAGMENTS double,UPPERLAYER_IP_DL_FRAGMENTS double,DOWNLAYER_IP_UL_FRAGMENTS double,DOWNLAYER_IP_DL_FRAGMENTS double,VALID_TRANS_DURATION double,AIR_PORT_DURATION double,RADIO_CONN_TIMES double,RAN_NE_ID double,AVG_UL_RTT double,AVG_DW_RTT double,UL_RTT_LONG_NUM int,DW_RTT_LONG_NUM int,UL_RTT_STAT_NUM int,DW_RTT_STAT_NUM int,USER_PROBE_UL_LOST_PKT int,SERVER_PROBE_UL_LOST_PKT int,SERVER_PROBE_DW_LOST_PKT int,USER_PROBE_DW_LOST_PKT int,CHARGING_CHARACTERISTICS double,DL_SERIOUS_OUT_OF_ORDER_NUM double,DL_SLIGHT_OUT_OF_ORDER_NUM double,DL_FLIGHT_TOTAL_SIZE double,DL_FLIGHT_TOTAL_NUM double,DL_MAX_FLIGHT_SIZE double,UL_SERIOUS_OUT_OF_ORDER_NUM double,UL_SLIGHT_OUT_OF_ORDER_NUM double,UL_FLIGHT_TOTAL_SIZE double,UL_FLIGHT_TOTAL_NUM double,UL_MAX_FLIGHT_SIZE double,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS double,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS double,DL_CONTINUOUS_RETRANSMISSION_DELAY double,USER_HUNGRY_DELAY double,SERVER_HUNGRY_DELAY double,AVG_DW_RTT_MICRO_SEC int,AVG_UL_RTT_MICRO_SEC int,FLOW_SAMPLE_RATIO int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,IMSI,IMEI,MS_IP,SERVER_IP,HOST,SP,MS_INDICATOR,streaming_url','DICTIONARY_INCLUDE'='SESSION_INDICATOR,SERVER_DECIMAL,TCP_STATES_BIGINTPOSITIVE')")

+    sql("create table smart_500_DE (MSISDN string,IMSI string,IMEI string,INTERFACEID int,GROUPID int,GGSN_ID double,SGSN_ID double,SESSION_INDICATOR int,BEGIN_TIME double,BEGIN_TIME_MSEL int,END_TIME double,END_TIME_MSEL int,PROT_CATEGORY int,PROT_TYPE int,L7_CARRIER_PROT int,SUB_PROT_TYPE int,SID double,PROBEID double,ENCRYPT_VERSION int,ROAMING_TYPE int,ROAM_DIRECTION int,MS_IP string,SERVER_IP string,MS_PORT int,SERVER_DECIMAL Decimal,APN string,SGSN_SIG_IP string,GGSN_SIG_IP_BigInt_NEGATIVE bigint,SGSN_USER_IP string,GGSN_USER_IP string,MCC string,MNC string,RAT int,LAC string,RAC string,SAC string,CI string,BROWSER_TIMESTAMP timestamp,TCP_CONN_STATES int,TCP_STATES_BIGINTPOSITIVE int,TCP_WIN_SIZE int,TCP_MSS int,TCP_CONN_TIMES int,TCP_CONN_2_FAILED_TIMES int,TCP_CONN_3_FAILED_TIMES int,HOST string,STREAMING_URL string,GET_STREAMING_FAILED_CODE int,GET_STREAMING_FLAG int,GET_NUM int,GET_SUCCEED_NUM int,GET_RETRANS_NUM int,GET_TIMEOUT_NUM int,INTBUFFER_FST_FLAG int,INTBUFFER_FULL_FLAG int,STALL_NUM int,VIDEO_FRAME_RATE int,VIDEO_CODEC_ID string,VIDEO_WIDTH int,VIDEO_HEIGHT int,AUDIO_CODEC_ID string,MEDIA_FILE_TYPE int,PLAY_STATE int,STREAMING_FLAG int,TCP_STATUS_INDICATOR int,DISCONNECTION_FLAG int,FAILURE_CODE int,FLAG int,TAC string,ECI string,TCP_SYN_TIME_MSEL int,TCP_FST_SYN_DIRECTION int,RAN_NE_USER_IP string,HOMEMCC string,HOMEMNC string,CHARGE_FLAG int,PREPAID_FLAG int,USER_AGENT string,MS_WIN_STAT_TOTAL_NUM int,MS_WIN_STAT_SMALL_NUM int,MS_ACK_TO_1STGET_DELAY int,SERVER_ACK_TO_1STDATA_DELAY int,STREAMING_TYPE int,SOURCE_VIDEO_QUALITY int,TETHERING_FLAG int,CARRIER_ID double,LAYER1ID int,LAYER2ID int,LAYER3ID int,LAYER4ID int,LAYER5ID int,LAYER6ID int,CHARGING_RULE_BASE_NAME string,SP string,EXTENDED_URL string,SV string,FIRST_SAI_CGI_ECGI string,EXTENDED_URL_OTHER string,SIGNALING_USE_FLAG int,DNS_RETRANS_NUM int,DNS_FAIL_CODE int,FIRST_RAT int,MS_INDICATOR string,LAST_SAI_CGI_ECGI string,LAST_RAT int,FIRST_LONGITUDE double,FIRST_LATITUDE double,FIRST_ALTITUDE int,FIRST_RASTERLONGITUDE double,FIRST_RASTERLATITUDE double,FIRST_RASTERALTITUDE int,FIRST_FREQUENCYSPOT int,FIRST_CLUTTER int,FIRST_USERBEHAVIOR int,FIRST_SPEED int,FIRST_CREDIBILITY int,LAST_LONGITUDE double,LAST_LATITUDE double,LAST_ALTITUDE int,LAST_RASTERLONGITUDE double,LAST_RASTERLATITUDE double,LAST_RASTERALTITUDE int,LAST_FREQUENCYSPOT int,LAST_CLUTTER int,LAST_USERBEHAVIOR int,LAST_SPEED int,LAST_CREDIBILITY int,IMEI_CIPHERTEXT string,APP_ID int,DOMAIN_NAME string,STREAMING_CACHE_IP string,STOP_LONGER_THAN_MIN_THRESHOLD int,STOP_LONGER_THAN_MAX_THRESHOLD int,PLAY_END_STAT int,STOP_START_TIME1 double,STOP_END_TIME1 double,STOP_START_TIME2 double,STOP_END_TIME2 double,STOP_START_TIME3 double,STOP_END_TIME3 double,STOP_START_TIME4 double,STOP_END_TIME4 double,STOP_START_TIME5 double,STOP_END_TIME5 double,STOP_START_TIME6 double,STOP_END_TIME6 double,STOP_START_TIME7 double,STOP_END_TIME7 double,STOP_START_TIME8 double,STOP_END_TIME8 double,STOP_START_TIME9 double,STOP_END_TIME9 double,STOP_START_TIME10 double,STOP_END_TIME10 double,FAIL_CLASS double,RECORD_TYPE double,NODATA_COUNT double,VIDEO_NODATA_DURATION double,VIDEO_SMOOTH_DURATION double,VIDEO_SD_DURATION double,VIDEO_HD_DURATION double,VIDEO_UHD_DURATION double,VIDEO_FHD_DURATION double,FLUCTUATION double,START_DOWNLOAD_THROUGHPUT double,L7_UL_GOODPUT_FULL_MSS double,SESSIONKEY string,FIRST_UCELLID double,LAST_UCELLID double,UCELLID1 double,LONGITUDE1 double,LATITUDE1 double,UCELLID2 double,LONGITUDE2 double,LATITUDE2 double,UCELLID3 double,LONGITUDE3 double,LATITUDE3 double,UCELLID4 double,LONGITUDE4 double,LATITUDE4 double,UCELLID5 double,LONGITUDE5 double,LATITUDE5 double,UCELLID6 double,LONGITUDE6 double,LATITUDE6 double,UCELLID7 double,LONGITUDE7 double,LATITUDE7 double,UCELLID8 double,LONGITUDE8 double,LATITUDE8 double,UCELLID9 double,LONGITUDE9 double,LATITUDE9 double,UCELLID10 double,LONGITUDE10 double,LATITUDE10 double,INTBUFFER_FULL_DELAY double,STALL_DURATION double,STREAMING_DW_PACKETS double,STREAMING_DOWNLOAD_DELAY double,PLAY_DURATION double,STREAMING_QUALITY int,VIDEO_DATA_RATE double,AUDIO_DATA_RATE double,STREAMING_FILESIZE double,STREAMING_DURATIOIN double,TCP_SYN_TIME double,TCP_RTT_STEP1 double,CHARGE_ID double,UL_REVERSE_TO_DL_DELAY double,DL_REVERSE_TO_UL_DELAY double,DATATRANS_DW_GOODPUT double,DATATRANS_DW_TOTAL_DURATION double,SUM_FRAGMENT_INTERVAL double,TCP_FIN_TIMES double,TCP_RESET_TIMES double,URL_CLASSIFICATION double,STREAMING_LQ_DURATIOIN double,MAX_DNS_DELAY double,MAX_DNS2SYN double,MAX_LATANCY_OF_LINK_SETUP double,MAX_SYNACK2FIRSTACK double,MAX_SYNACK2LASTACK double,MAX_ACK2GET_DELAY double,MAX_FRAG_INTERVAL_PREDELAY double,SUM_FRAG_INTERVAL_PREDELAY double,SERVICE_DELAY_MSEC double,HOMEPROVINCE double,HOMECITY double,SERVICE_ID double,CHARGING_CLASS double,DATATRANS_UL_DURATION double,ASSOCIATED_ID double,PACKET_LOSS_NUM double,JITTER double,MS_DNS_DELAY_MSEL double,GET_STREAMING_DELAY double,TCP_UL_RETRANS_WITHOUTPL double,TCP_DW_RETRANS_WITHOUTPL double,GET_MAX_UL_SIZE double,GET_MIN_UL_SIZE double,GET_MAX_DL_SIZE double,GET_MIN_DL_SIZE double,L4_UL_THROUGHPUT double,L4_DW_THROUGHPUT double,L4_UL_GOODPUT double,L4_DW_GOODPUT double,NETWORK_UL_TRAFFIC double,NETWORK_DL_TRAFFIC double,L4_UL_PACKETS double,L4_DW_PACKETS double,TCP_RTT double,TCP_UL_OUTOFSEQU double,TCP_DW_OUTOFSEQU double,TCP_UL_RETRANS double,TCP_DW_RETRANS double,TCP_UL_RETRANS_WITHPL double,TCP_DW_RETRANS_WITHPL double,TCP_UL_PACKAGES_WITHPL double,TCP_DW_PACKAGES_WITHPL double,TCP_UL_PACKAGES_WITHOUTPL double,TCP_DW_PACKAGES_WITHOUTPL double,UPPERLAYER_IP_UL_PACKETS double,UPPERLAYER_IP_DL_PACKETS double,DOWNLAYER_IP_UL_PACKETS double,DOWNLAYER_IP_DL_PACKETS double,UPPERLAYER_IP_UL_FRAGMENTS double,UPPERLAYER_IP_DL_FRAGMENTS double,DOWNLAYER_IP_UL_FRAGMENTS double,DOWNLAYER_IP_DL_FRAGMENTS double,VALID_TRANS_DURATION double,AIR_PORT_DURATION double,RADIO_CONN_TIMES double,RAN_NE_ID double,AVG_UL_RTT double,AVG_DW_RTT double,UL_RTT_LONG_NUM int,DW_RTT_LONG_NUM int,UL_RTT_STAT_NUM int,DW_RTT_STAT_NUM int,USER_PROBE_UL_LOST_PKT int,SERVER_PROBE_UL_LOST_PKT int,SERVER_PROBE_DW_LOST_PKT int,USER_PROBE_DW_LOST_PKT int,CHARGING_CHARACTERISTICS double,DL_SERIOUS_OUT_OF_ORDER_NUM double,DL_SLIGHT_OUT_OF_ORDER_NUM double,DL_FLIGHT_TOTAL_SIZE double,DL_FLIGHT_TOTAL_NUM double,DL_MAX_FLIGHT_SIZE double,UL_SERIOUS_OUT_OF_ORDER_NUM double,UL_SLIGHT_OUT_OF_ORDER_NUM double,UL_FLIGHT_TOTAL_SIZE double,UL_FLIGHT_TOTAL_NUM double,UL_MAX_FLIGHT_SIZE double,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS double,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS double,DL_CONTINUOUS_RETRANSMISSION_DELAY double,USER_HUNGRY_DELAY double,SERVER_HUNGRY_DELAY double,AVG_DW_RTT_MICRO_SEC int,AVG_UL_RTT_MICRO_SEC int,FLOW_SAMPLE_RATIO int) STORED BY 'org.apache.carbondata.format' ")

     try {

       sql(s"LOAD DATA INPATH '$resourcesPath/seq_20Records.csv' into table smart_500_DE options('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='SID,PROBEID,INTERFACEID,GROUPID,GGSN_ID,SGSN_ID,dummy,SESSION_INDICATOR,BEGIN_TIME,BEGIN_TIME_MSEL,END_TIME,END_TIME_MSEL,PROT_CATEGORY,PROT_TYPE,L7_CARRIER_PROT,SUB_PROT_TYPE,MSISDN,IMSI,IMEI,ENCRYPT_VERSION,ROAMING_TYPE,ROAM_DIRECTION,MS_IP,SERVER_IP,MS_PORT,APN,SGSN_SIG_IP,GGSN_USER_IP,SGSN_USER_IP,MCC,MNC,RAT,LAC,RAC,SAC,CI,SERVER_DECIMAL,BROWSER_TIMESTAMP,TCP_CONN_STATES,GGSN_SIG_IP_BigInt_NEGATIVE,TCP_STATES_BIGINTPOSITIVE,dummy,TCP_WIN_SIZE,dummy,TCP_MSS,dummy,TCP_CONN_TIMES,dummy,TCP_CONN_2_FAILED_TIMES,dummy,TCP_CONN_3_FAILED_TIMES,HOST,STREAMING_URL,dummy,GET_STREAMING_FAILED_CODE,dummy,GET_STREAMING_FLAG,dummy,GET_NUM,dummy,GET_SUCCEED_NUM,dummy,GET_RETRANS_NUM,dummy,GET_TIMEOUT_NUM,INTBUFFER_FST_FLAG,INTBUFFER_FULL_FLAG,STALL_NUM,dummy,VIDEO_FRAME_RATE,dummy,VIDEO_CODEC_ID,dummy,VIDEO_WIDTH,dummy,VIDEO_HEIGHT,dummy,AUDIO_CODEC_ID,dummy,MEDIA_FILE_TYPE,dummy,PLAY_STATE,dummy,PLAY_STATE,dummy,STREAMING_FLAG,dummy,TCP_STATUS_INDICATOR,dummy,DISCONNECTION_FLAG,dummy,FAILURE_CODE,FLAG,TAC,ECI,dummy,TCP_SYN_TIME_MSEL,dummy,TCP_FST_SYN_DIRECTION,RAN_NE_USER_IP,HOMEMCC,HOMEMNC,dummy,CHARGE_FLAG,dummy,PREPAID_FLAG,dummy,USER_AGENT,dummy,MS_WIN_STAT_TOTAL_NUM,dummy,MS_WIN_STAT_SMALL_NUM,dummy,MS_ACK_TO_1STGET_DELAY,dummy,SERVER_ACK_TO_1STDATA_DELAY,dummy,STREAMING_TYPE,dummy,SOURCE_VIDEO_QUALITY,TETHERING_FLAG,CARRIER_ID,LAYER1ID,LAYER2ID,dummy,LAYER3ID,dummy,LAYER4ID,dummy,LAYER5ID,dummy,LAYER6ID,CHARGING_RULE_BASE_NAME,SP,dummy,EXTENDED_URL,SV,FIRST_SAI_CGI_ECGI,dummy,EXTENDED_URL_OTHER,SIGNALING_USE_FLAG,dummy,DNS_RETRANS_NUM,dummy,DNS_FAIL_CODE,FIRST_RAT,FIRST_RAT,MS_INDICATOR,LAST_SAI_CGI_ECGI,LAST_RAT,dummy,FIRST_LONGITUDE,dummy,FIRST_LATITUDE,dummy,FIRST_ALTITUDE,dummy,FIRST_RASTERLONGITUDE,dummy,FIRST_RASTERLATITUDE,dummy,FIRST_RASTERALTITUDE,dummy,FIRST_FREQUENCYSPOT,dummy,FIRST_CLUTTER,dummy,FIRST_USERBEHAVIOR,dummy,FIRST_SPEED,dummy,FIRST_CREDIBILITY,dummy,LAST_LONGITUDE,dummy,LAST_LATITUDE,dummy,LAST_ALTITUDE,dummy,LAST_RASTERLONGITUDE,dummy,LAST_RASTERLATITUDE,dummy,LAST_RASTERALTITUDE,dummy,LAST_FREQUENCYSPOT,dummy,LAST_CLUTTER,dummy,LAST_USERBEHAVIOR,dummy,LAST_SPEED,dummy,LAST_CREDIBILITY,dummy,IMEI_CIPHERTEXT,APP_ID,dummy,DOMAIN_NAME,dummy,STREAMING_CACHE_IP,dummy,STOP_LONGER_THAN_MIN_THRESHOLD,dummy,STOP_LONGER_THAN_MAX_THRESHOLD,dummy,PLAY_END_STAT,dummy,STOP_START_TIME1,dummy,STOP_END_TIME1,dummy,STOP_START_TIME2,dummy,STOP_END_TIME2,dummy,STOP_START_TIME3,dummy,STOP_END_TIME3,dummy,STOP_START_TIME4,dummy,STOP_END_TIME4,dummy,STOP_START_TIME5,dummy,STOP_END_TIME5,dummy,STOP_START_TIME6,dummy,STOP_END_TIME6,dummy,STOP_START_TIME7,dummy,STOP_END_TIME7,dummy,STOP_START_TIME8,dummy,STOP_END_TIME8,dummy,STOP_START_TIME9,dummy,STOP_END_TIME9,dummy,STOP_START_TIME10,dummy,STOP_END_TIME10,dummy,FAIL_CLASS,RECORD_TYPE,dummy,NODATA_COUNT,dummy,VIDEO_NODATA_DURATION,dummy,VIDEO_SMOOTH_DURATION,dummy,VIDEO_SD_DURATION,dummy,VIDEO_HD_DURATION,dummy,VIDEO_UHD_DURATION,dummy,VIDEO_FHD_DURATION,dummy,FLUCTUATION,dummy,START_DOWNLOAD_THROUGHPUT,dummy,L7_UL_GOODPUT_FULL_MSS,dummy,SESSIONKEY,dummy,FIRST_UCELLID,dummy,LAST_UCELLID,dummy,UCELLID1,dummy,LONGITUDE1,dummy,LATITUDE1,dummy,UCELLID2,dummy,LONGITUDE2,dummy,LATITUDE2,dummy,UCELLID3,dummy,LONGITUDE3,dummy,LATITUDE3,dummy,UCELLID4,dummy,LONGITUDE4,dummy,LATITUDE4,dummy,UCELLID5,dummy,LONGITUDE5,dummy,LATITUDE5,dummy,UCELLID6,dummy,LONGITUDE6,dummy,LATITUDE6,dummy,UCELLID7,dummy,LONGITUDE7,dummy,LATITUDE7,dummy,UCELLID8,dummy,LONGITUDE8,dummy,LATITUDE8,dummy,UCELLID9,dummy,LONGITUDE9,dummy,LATITUDE9,dummy,UCELLID10,dummy,LONGITUDE10,dummy,LATITUDE10,dummy,INTBUFFER_FULL_DELAY,dummy,STALL_DURATION,dummy,STREAMING_DW_PACKETS,dummy,STREAMING_DOWNLOAD_DELAY,dummy,PLAY_DURATION,dummy,STREAMING_QUALITY,dummy,VIDEO_DATA_RATE,dummy,AUDIO_DATA_RATE,dummy,STREAMING_FILESIZE,dummy,STREAMING_DURATIOIN,dummy,TCP_SYN_TIME,dummy,TCP_RTT_STEP1,CHARGE_ID,dummy,UL_REVERSE_TO_DL_DELAY,dummy,DL_REVERSE_TO_UL_DELAY,dummy,DATATRANS_DW_GOODPUT,dummy,DATATRANS_DW_TOTAL_DURATION,dummy,SUM_FRAGMENT_INTERVAL,dummy,TCP_FIN_TIMES,dummy,TCP_RESET_TIMES,dummy,URL_CLASSIFICATION,dummy,STREAMING_LQ_DURATIOIN,dummy,MAX_DNS_DELAY,dummy,MAX_DNS2SYN,dummy,MAX_LATANCY_OF_LINK_SETUP,dummy,MAX_SYNACK2FIRSTACK,dummy,MAX_SYNACK2LASTACK,dummy,MAX_ACK2GET_DELAY,dummy,MAX_FRAG_INTERVAL_PREDELAY,dummy,SUM_FRAG_INTERVAL_PREDELAY,dummy,SERVICE_DELAY_MSEC,dummy,HOMEPROVINCE,dummy,HOMECITY,dummy,SERVICE_ID,dummy,CHARGING_CLASS,dummy,DATATRANS_UL_DURATION,dummy,ASSOCIATED_ID,dummy,PACKET_LOSS_NUM,dummy,JITTER,dummy,MS_DNS_DELAY_MSEL,dummy,GET_STREAMING_DELAY,dummy,TCP_UL_RETRANS_WITHOUTPL,dummy,TCP_DW_RETRANS_WITHOUTPL,dummy,GET_MAX_UL_SIZE,dummy,GET_MIN_UL_SIZE,dummy,GET_MAX_DL_SIZE,dummy,GET_MIN_DL_SIZE,dummy,FLOW_SAMPLE_RATIO,dummy,UL_RTT_LONG_NUM,dummy,DW_RTT_LONG_NUM,dummy,UL_RTT_STAT_NUM,dummy,DW_RTT_STAT_NUM,dummy,USER_PROBE_UL_LOST_PKT,dummy,SERVER_PROBE_UL_LOST_PKT,dummy,SERVER_PROBE_DW_LOST_PKT,dummy,USER_PROBE_DW_LOST_PKT,dummy,AVG_DW_RTT_MICRO_SEC,dummy,AVG_UL_RTT_MICRO_SEC,dummy,RAN_NE_ID,dummy,AVG_UL_RTT,dummy,AVG_DW_RTT,dummy,CHARGING_CHARACTERISTICS,dummy,DL_SERIOUS_OUT_OF_ORDER_NUM,dummy,DL_SLIGHT_OUT_OF_ORDER_NUM,dummy,DL_FLIGHT_TOTAL_SIZE,dummy,DL_FLIGHT_TOTAL_NUM,dummy,DL_MAX_FLIGHT_SIZE,dummy,VALID_TRANS_DURATION,dummy,AIR_PORT_DURATION,dummy,RADIO_CONN_TIMES,dummy,UL_SERIOUS_OUT_OF_ORDER_NUM,dummy,UL_SLIGHT_OUT_OF_ORDER_NUM,dummy,UL_FLIGHT_TOTAL_SIZE,dummy,UL_FLIGHT_TOTAL_NUM,dummy,UL_MAX_FLIGHT_SIZE,dummy,USER_DL_SLIGHT_OUT_OF_ORDER_PACKETS,dummy,SERVER_UL_SLIGHT_OUT_OF_ORDER_PACKETS,dummy,DL_CONTINUOUS_RETRANSMISSION_DELAY,dummy,USER_HUNGRY_DELAY,dummy,SERVER_HUNGRY_DELAY,dummy,UPPERLAYER_IP_UL_FRAGMENTS,dummy,UPPERLAYER_IP_DL_FRAGMENTS,dummy,DOWNLAYER_IP_UL_FRAGMENTS,dummy,DOWNLAYER_IP_DL_FRAGMENTS,dummy,UPPERLAYER_IP_UL_PACKETS,dummy,UPPERLAYER_IP_DL_PACKETS,dummy,DOWNLAYER_IP_UL_PACKETS,dummy,DOWNLAYER_IP_DL_PACKETS,dummy,TCP_UL_PACKAGES_WITHPL,dummy,TCP_DW_PACKAGES_WITHPL,dummy,TCP_UL_PACKAGES_WITHOUTPL,dummy,TCP_DW_PACKAGES_WITHOUTPL,dummy,TCP_UL_RETRANS_WITHPL,dummy,TCP_DW_RETRANS_WITHPL,L4_UL_THROUGHPUT,L4_DW_THROUGHPUT,L4_UL_GOODPUT,L4_DW_GOODPUT,NETWORK_UL_TRAFFIC,NETWORK_DL_TRAFFIC,L4_UL_PACKETS,L4_DW_PACKETS,TCP_RTT,TCP_UL_OUTOFSEQU,TCP_DW_OUTOFSEQU,TCP_UL_RETRANS,TCP_DW_RETRANS')")

       assert(true)

diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
index 201da39..3a3881b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
@@ -52,8 +52,6 @@
            (ID Int, date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int)
            STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_EXCLUDE'='country,phonetype,serialname',
-           'DICTIONARY_INCLUDE'='ID')
         """)
     } catch {
       case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 4a0e74f..2aad2f3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -125,18 +125,6 @@
     checkAnswer(sql("SELECT COUNT(*) FROM carbon_globalsort"), Seq(Row(11)))
   }
 
-  // ----------------------------------- Single Pass -----------------------------------
-  // Waiting for merge [CARBONDATA-1145]
-  test("Test GLOBAL_SORT with SINGLE_PASS") {
-    sql(s"LOAD DATA LOCAL INPATH '$filePath' INTO TABLE carbon_globalsort " +
-      "OPTIONS('SINGLE_PASS'='TRUE')")
-
-    assert(getIndexFileCount("carbon_globalsort") === 2)
-    checkAnswer(sql("SELECT COUNT(*) FROM carbon_globalsort"), Seq(Row(12)))
-    checkAnswer(sql("SELECT * FROM carbon_globalsort ORDER BY name"),
-      sql("SELECT * FROM carbon_localsort_once ORDER BY name"))
-  }
-
   // ----------------------------------- Configuration Validity -----------------------------------
 
   test("Number of partitions should be greater than 0") {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
index 867fdef..9b40a5f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
@@ -166,76 +166,6 @@
       sql("SELECT count(*) FROM carbon2"),Seq(Row(32000)))
   }
 
-  test("test load dataframe with integer columns included in the dictionary"){
-    df2.write
-      .format("carbondata")
-      .option("tableName", "carbon5")
-      .option("compress", "true")
-      .option("dictionary_include","c3,c4")
-      .mode(SaveMode.Overwrite)
-      .save()
-    checkAnswer(
-      sql("select count(*) from carbon5 where c3 > 300"), Row(700)
-    )
-  }
-
-  test("test load dataframe with string column excluded from the dictionary"){
-    df2.write
-      .format("carbondata")
-      .option("tableName", "carbon6")
-      .option("compress", "true")
-      .option("dictionary_exclude","c2")
-      .mode(SaveMode.Overwrite)
-      .save()
-    checkAnswer(
-      sql("select count(*) from carbon6 where c3 > 300"), Row(700)
-    )
-  }
-
-  test("test load dataframe with both dictionary include and exclude specified"){
-    df2.write
-      .format("carbondata")
-      .option("tableName", "carbon7")
-      .option("compress", "true")
-      .option("dictionary_include","c3,c4")
-      .option("dictionary_exclude","c2")
-      .mode(SaveMode.Overwrite)
-      .save()
-    checkAnswer(
-      sql("select count(*) from carbon7 where c3 > 300"), Row(700)
-    )
-  }
-
-  test("test load dataframe with single pass enabled") {
-    // save dataframe to carbon file
-    df2.write
-      .format("carbondata")
-      .option("tableName", "carbon8")
-      .option("tempCSV", "false")
-      .option("single_pass", "true")
-      .option("compress", "false")
-      .mode(SaveMode.Overwrite)
-      .save()
-    checkAnswer(
-      sql("select count(*) from carbon8 where c3 > 500"), Row(500)
-    )
-  }
-
-  test("test load dataframe with single pass disabled") {
-    // save dataframe to carbon file
-    df2.write
-      .format("carbondata")
-      .option("tableName", "carbon9")
-      .option("tempCSV", "true")
-      .option("single_pass", "false")
-      .option("compress", "false")
-      .mode(SaveMode.Overwrite)
-      .save()
-    checkAnswer(
-      sql("select count(*) from carbon9 where c3 > 500"), Row(500)
-    )
-  }
-
   test("test datasource table with specified table path") {
     val path = "./source"
     df2.write
@@ -260,7 +190,6 @@
       .format("carbondata")
       .option("tableName", "carbon11")
       .option("tempCSV", "true")
-      .option("single_pass", "false")
       .option("compress", "false")
       .option("streaming", "true")
       .mode(SaveMode.Overwrite)
@@ -299,7 +228,6 @@
       .format("carbondata")
       .option("tableName", tableName)
       .option("tempCSV", "false")
-      .option("single_pass", "false")
       .option("table_blocksize", "256")
       .option("compress", "false")
       .mode(SaveMode.Overwrite)
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
index 52edf3a..1a4208e 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
@@ -30,25 +30,9 @@
            (ID Int, date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int)
            STORED BY 'carbondata'
-           TBLPROPERTIES('dictionary_include'='country,name,phonetype,serialname')
            """)
   }
 
-  test("test load data use all dictionary, and given wrong format dictionary values") {
-    try {
-      sql(s"""
-           LOAD DATA LOCAL INPATH '$resourcesPath/source_without_header.csv' into table t3
-           options('FILEHEADER'='id,date,country,name,phonetype,serialname,salary',
-           'All_DICTIONARY_PATH'='$resourcesPath/dict.txt','single_pass'='true')
-           """)
-      assert(false)
-    } catch {
-      case e: DataLoadingException =>
-        assert(e.getMessage.equals("Data Loading failure, dictionary values are " +
-          "not in correct format!"))
-    }
-  }
-
   override def afterAll {
     sql("DROP TABLE IF EXISTS t3")
   }
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
index 134ac76..f6cb1b5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
@@ -122,9 +122,7 @@
       "CREATE table testtable (empno string, empname String, designation String, doj String, " +
         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
         "projectcode string, projectjoindate String, projectenddate String,attendance double," +
-        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
-        "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
+        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       "create table testhivetable(empno string, empname String, designation string, doj String, " +
@@ -165,9 +163,7 @@
       "CREATE table testtable1 (empno string, empname String, designation String, doj String, " +
         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
         "projectcode string, projectjoindate String, projectenddate String,attendance double," +
-        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
-        "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
+        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       "create table testhivetable1(empno string, empname String, designation string, doj String, " +
@@ -236,8 +232,7 @@
       "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
       "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
       "double,contractNumber double) " +
-      "STORED BY 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')"
+      "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdata.csv' INTO table " +
@@ -260,8 +255,7 @@
       "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
       "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
       "double,contractNumber double) " +
-      "STORED BY 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId','DICTIONARY_EXCLUDE'='channelsId')"
+      "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complextypediffentcolheaderorder.csv' INTO " +
@@ -330,8 +324,7 @@
       "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
       "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
       "double,contractNumber double) " +
-      "STORED BY 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')"
+      "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdatastructextra.csv' INTO table " +
@@ -351,9 +344,7 @@
       "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
       "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
       "double,contractNumber double) " +
-      "STORED BY 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId', 'DICTIONARY_EXCLUDE'='ROMSize," +
-      "purchasedate')"
+      "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdata.csv' INTO table " +
@@ -368,9 +359,7 @@
   test("test Complex Data type - Array and Struct of timestamp with dictionary include") {
     sql("DROP TABLE IF EXISTS array_timestamp")
     sql(
-      "create table array_timestamp (date1 array<timestamp>,date2 struct<date:timestamp> ) stored" +
-      " by 'carbondata' tblproperties" +
-      "('dictionary_include'='date1,date2')")
+      "create table array_timestamp (date1 array<timestamp>,date2 struct<date:timestamp> ) stored by 'carbondata' ")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
     sql("insert into array_timestamp values(array('2015-01-01 00:00:00','2016-01-01 00:00:00'),named_struct('date','2017-01-01 00:00:00'))")
@@ -385,15 +374,13 @@
   test("array<string> and string datatype for same column is not working properly") {
     sql("drop table if exists complexcarbontable")
     sql("create table complexcarbontable(deviceInformationId int, MAC array<string>, channelsId string, "+
-        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' "+
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')")
+        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA local inpath '$resourcesPath/complexdatareordered.csv' INTO table complexcarbontable "+
         "OPTIONS('DELIMITER'=',', 'QUOTECHAR'='\"', 'FILEHEADER'='deviceInformationId,MAC,channelsId,ROMSize,purchasedate,gamePointId,contractNumber',"+
         "'COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
     sql("drop table if exists complexcarbontable")
     sql("create table primitivecarbontable(deviceInformationId int, MAC string, channelsId string, "+
-        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' "+
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')")
+        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA local inpath '$resourcesPath/complexdatareordered.csv' INTO table primitivecarbontable "+
         "OPTIONS('DELIMITER'=',', 'QUOTECHAR'='\"', 'FILEHEADER'='deviceInformationId,MAC,channelsId,ROMSize,purchasedate,gamePointId,contractNumber',"+
         "'COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
@@ -583,7 +570,7 @@
            (ID decimal(5,5), date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int, complex
            array<decimal(4,2)>)
-           STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='complex')
+           STORED BY 'org.apache.carbondata.format'
       """
     )
 
@@ -605,7 +592,7 @@
            (ID decimal(5,5), date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int, complex
            struct<a:decimal(4,2)>)
-           STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='complex')
+           STORED BY 'org.apache.carbondata.format'
       """
     )
 
@@ -628,7 +615,7 @@
            (ID decimal, date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int, complex
            array<struct<a:decimal(4,2),str:string>>)
-           STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='complex')
+           STORED BY 'org.apache.carbondata.format'
       """
     )
     sql(
@@ -658,9 +645,7 @@
       "CREATE table carbontable1 (empno string, empname String, designation String, doj String, " +
         "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
         "projectcode string, projectjoindate String, projectenddate String,attendance double," +
-        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
-        "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
+        "utilization double,salary double) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       "create table hivetable1 (empno string, empname String, designation string, doj String, " +
@@ -695,10 +680,7 @@
 
   test("test data load with double datatype") {
     sql("drop table if exists double_test")
-    sql(
-      "CREATE table double_test (empno string, salary double) STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='empno')"
-    )
+    sql("CREATE table double_test (empno string, salary double) STORED BY 'carbondata' ")
     sql(
       s"load data local inpath '$resourcesPath/double.csv' into table double_test options" +
       "('FILEHEADER'='empno,salary')")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxUnsafe.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxUnsafe.scala
index de01092..20d781b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxUnsafe.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxUnsafe.scala
@@ -17,8 +17,6 @@
 
 package org.apache.carbondata.spark.testsuite.dataload
 
-import java.io.File
-
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -132,9 +130,7 @@
       "CREATE table testtable (empno string, empname String, designation String, doj String, " +
           "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
           "projectcode string, projectjoindate String, projectenddate String,attendance double," +
-          "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-          "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
-          "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
+          "utilization double,salary double) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       "create table testhivetable(empno string, empname String, designation string, doj String, " +
@@ -175,9 +171,7 @@
       "CREATE table testtable1 (empno string, empname String, designation String, doj String, " +
           "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
           "projectcode string, projectjoindate String, projectenddate String,attendance double," +
-          "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-          "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
-          "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
+          "utilization double,salary double) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       "create table testhivetable1(empno string, empname String, designation string, doj String, " +
@@ -246,8 +240,7 @@
         "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
         "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
         "double,contractNumber double) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdata.csv' INTO table " +
@@ -270,8 +263,7 @@
         "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
         "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
         "double,contractNumber double) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId','DICTIONARY_EXCLUDE'='channelsId')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complextypediffentcolheaderorder.csv' INTO " +
@@ -340,8 +332,7 @@
         "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
         "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
         "double,contractNumber double) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdatastructextra.csv' INTO table " +
@@ -361,9 +352,7 @@
         "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
         "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
         "double,contractNumber double) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId', 'DICTIONARY_EXCLUDE'='ROMSize," +
-        "purchasedate')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA local inpath '$resourcesPath/complexdata.csv' INTO table " +
@@ -378,15 +367,13 @@
   test("array<string> and string datatype for same column is not working properly") {
     sql("drop table if exists complexcarbontable")
     sql("create table complexcarbontable(deviceInformationId int, MAC array<string>, channelsId string, "+
-        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' "+
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')")
+        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA local inpath '$resourcesPath/complexdatareordered.csv' INTO table complexcarbontable "+
         "OPTIONS('DELIMITER'=',', 'QUOTECHAR'='\"', 'FILEHEADER'='deviceInformationId,MAC,channelsId,ROMSize,purchasedate,gamePointId,contractNumber',"+
         "'COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
     sql("drop table if exists complexcarbontable")
     sql("create table primitivecarbontable(deviceInformationId int, MAC string, channelsId string, "+
-        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' "+
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')")
+        "ROMSize string, purchasedate string, gamePointId double,contractNumber double) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA local inpath '$resourcesPath/complexdatareordered.csv' INTO table primitivecarbontable "+
         "OPTIONS('DELIMITER'=',', 'QUOTECHAR'='\"', 'FILEHEADER'='deviceInformationId,MAC,channelsId,ROMSize,purchasedate,gamePointId,contractNumber',"+
         "'COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
@@ -576,7 +563,7 @@
            (ID decimal(5,5), date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int, complex
            array<decimal(4,2)>)
-           STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='complex')
+           STORED BY 'org.apache.carbondata.format'
       """
     )
 
@@ -598,7 +585,7 @@
            (ID decimal(5,5), date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int, complex
            struct<a:decimal(4,2)>)
-           STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='complex')
+           STORED BY 'org.apache.carbondata.format'
       """
     )
 
@@ -621,7 +608,7 @@
            (ID decimal, date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int, complex
            array<struct<a:decimal(4,2),str:string>>)
-           STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='complex')
+           STORED BY 'org.apache.carbondata.format'
       """
     )
     sql(
@@ -651,9 +638,7 @@
       "CREATE table carbontable1 (empno string, empname String, designation String, doj String, " +
           "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
           "projectcode string, projectjoindate String, projectenddate String,attendance double," +
-          "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
-          "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
-          "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
+          "utilization double,salary double) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       "create table hivetable1 (empno string, empname String, designation string, doj String, " +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
index 8793608..8c5383cf 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
@@ -35,78 +35,6 @@
     sql("drop table TestLoadTableOptions")
   }
 
-  def buildTableWithNoExistDictExclude() = {
-      sql(
-        """
-           CREATE TABLE IF NOT EXISTS t3
-           (ID Int, date Timestamp, country String,
-           name String, phonetype String, serialname String, salary Int)
-           STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_EXCLUDE'='country,phonetype,CCC')
-        """)
-  }
-
-  def buildTableWithNoExistDictInclude() = {
-      sql(
-        """
-           CREATE TABLE IF NOT EXISTS t3
-           (ID Int, date Timestamp, country String,
-           name String, phonetype String, serialname String, salary Int)
-           STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_INCLUDE'='AAA,country')
-        """)
-  }
-
-  def buildTableWithSameDictExcludeAndInclude() = {
-      sql(
-        """
-           CREATE TABLE IF NOT EXISTS t3
-           (ID Int, date Timestamp, country String,
-           name String, phonetype String, serialname String, salary Int)
-           STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_INCLUDE'='country','DICTIONARY_EXCLUDE'='country')
-        """)
-  }
-
-  def buildTableWithSameDictExcludeAndIncludeWithSpaces() = {
-    sql(
-      """
-           CREATE TABLE IF NOT EXISTS t3
-           (ID Int, date Timestamp, country String,
-           name String, phonetype String, serialname String, salary Int)
-           STORED BY 'org.apache.carbondata.format'
-           TBLPROPERTIES('DICTIONARY_INCLUDE'='country','DICTIONARY_EXCLUDE'='country ')
-      """)
-  }
-
-  test("test load data with dictionary exclude columns which no exist in table.") {
-    val e = intercept[MalformedCarbonCommandException] {
-      buildTableWithNoExistDictExclude()
-    }
-    assert(e.getMessage
-      .equals(
-        "DICTIONARY_EXCLUDE column: CCC does not exist in table or unsupported for complex child " +
-        "column. Please check the create table statement."))
-  }
-
-  test("test load data with dictionary include columns which no exist in table.") {
-    val e = intercept[MalformedCarbonCommandException] {
-      buildTableWithNoExistDictInclude()
-    }
-    assert(e.getMessage
-      .equals(
-        "DICTIONARY_INCLUDE column: AAA does not exist in table or unsupported for complex child " +
-        "column. Please check the create table statement."))
-  }
-
-  test("test load data with dictionary include is same with dictionary exclude") {
-    val e = intercept[MalformedCarbonCommandException] {
-      buildTableWithSameDictExcludeAndInclude()
-    }
-    assert(e.getMessage.equals("DICTIONARY_EXCLUDE can not contain the same column: country " +
-      "with DICTIONARY_INCLUDE. Please check the create table statement."))
-  }
-
   test("test load data with invalid option") {
     val e = intercept[MalformedCarbonCommandException] {
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE " +
@@ -134,11 +62,4 @@
     }
   }
 
-  test("test load data with dictionary include is same with dictionary exclude with spaces") {
-    val e = intercept[MalformedCarbonCommandException] {
-      buildTableWithSameDictExcludeAndIncludeWithSpaces()
-    }
-    assert(e.getMessage.equals("DICTIONARY_EXCLUDE can not contain the same column: country " +
-      "with DICTIONARY_INCLUDE. Please check the create table statement."))
-  }
 }
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNoMeasure.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNoMeasure.scala
index f1d25e6..3f7f352 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNoMeasure.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNoMeasure.scala
@@ -61,10 +61,7 @@
 
   test("test data loading with single no dictionary column") {
     sql("DROP TABLE IF EXISTS nomeasureTest_sd")
-    sql(
-      "CREATE TABLE nomeasureTest_sd (city String) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_EXCLUDE'='city')"
-    )
+    sql("CREATE TABLE nomeasureTest_sd (city String) STORED BY 'org.apache.carbondata.format' ")
     val testData = s"$resourcesPath/datasingleCol.csv"
     sql("LOAD DATA LOCAL INPATH '" + testData + "' into table nomeasureTest_sd options " +
       "('FILEHEADER'='city')"
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadWithSortTempCompressed.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadWithSortTempCompressed.scala
index 5d9f05c..3404c01 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadWithSortTempCompressed.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadWithSortTempCompressed.scala
@@ -73,7 +73,6 @@
       .format("carbondata")
       .option("tableName", simpleTable)
       .option("tempCSV", "true")
-      .option("DICTIONARY_INCLUDE", "c1,c2")
       .option("SORT_COLUMNS", "c1,c3")
       .save()
 
@@ -165,9 +164,7 @@
       "array<string>, locationinfo array<struct<ActiveAreaId:int, ActiveCountry:string, " +
       "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>, " +
       "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
-      "double,contractNumber double)  STORED BY 'org.apache.carbondata.format'  TBLPROPERTIES " +
-      "('DICTIONARY_INCLUDE'='deviceInformationId', 'DICTIONARY_EXCLUDE'='channelsId'," +
-      "'COLUMN_GROUP'='(ROMSize,ROMName)')")
+      "double,contractNumber double)  STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA local inpath '$resourcesPath/complextypesample.csv' INTO table" +
         s" $complexCarbonTable  OPTIONS('DELIMITER'=',', " +
         "'QUOTECHAR'='\"', 'FILEHEADER'='deviceInformationId,channelsId,ROMSize,ROMName," +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala
index 30d508d..530a8af 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestRangeColumnDataLoad.scala
@@ -412,70 +412,6 @@
     sql("DROP TABLE IF EXISTS carbon_range_column1")
   }
 
-  test("Test compaction for range_column - INT Datatype with Global Dict") {
-    deleteFile(filePath2)
-    createFile(filePath2, 10, 9)
-    deleteFile(filePath3)
-    createFile(filePath3, 10, 10)
-    sql("DROP TABLE IF EXISTS carbon_range_column1")
-    sql(
-      """
-        | CREATE TABLE carbon_range_column1(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_SCOPE'='LOCAL_SORT', 'SORT_COLUMNS'='name, city',
-        | 'range_column'='name', 'DICTIONARY_INCLUDE'='name')
-      """.stripMargin)
-
-    sql(s"LOAD DATA LOCAL INPATH '$filePath2' INTO TABLE carbon_range_column1 " +
-        "OPTIONS('HEADER'='false','GLOBAL_SORT_PARTITIONS'='3')")
-
-    sql(s"LOAD DATA LOCAL INPATH '$filePath3' INTO TABLE carbon_range_column1 " +
-        "OPTIONS('HEADER'='false','GLOBAL_SORT_PARTITIONS'='3')")
-
-    var res = sql("select * from carbon_range_column1").collect()
-
-    sql("ALTER TABLE carbon_range_column1 COMPACT 'MAJOR'")
-
-    checkAnswer(sql("select * from carbon_range_column1"), res)
-
-    deleteFile(filePath2)
-    deleteFile(filePath3)
-
-    sql("DROP TABLE IF EXISTS carbon_range_column1")
-  }
-
-  test("Test compaction for range_column - STRING Datatype with Global Dict") {
-    deleteFile(filePath2)
-    createFile(filePath2, 1000, 9)
-    deleteFile(filePath3)
-    createFile(filePath3, 10, 10)
-    sql("DROP TABLE IF EXISTS carbon_range_column1")
-    sql(
-      """
-        | CREATE TABLE carbon_range_column1(id INT, name STRING, city STRING, age INT)
-        | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('SORT_SCOPE'='LOCAL_SORT', 'SORT_COLUMNS'='name, city',
-        | 'range_column'='name', 'DICTIONARY_INCLUDE'='name')
-      """.stripMargin)
-
-    sql(s"LOAD DATA LOCAL INPATH '$filePath2' INTO TABLE carbon_range_column1 " +
-        "OPTIONS('HEADER'='false','GLOBAL_SORT_PARTITIONS'='3')")
-
-    sql(s"LOAD DATA LOCAL INPATH '$filePath3' INTO TABLE carbon_range_column1 " +
-        "OPTIONS('HEADER'='false','GLOBAL_SORT_PARTITIONS'='3')")
-
-    var res = sql("select * from carbon_range_column1").collect()
-
-    sql("ALTER TABLE carbon_range_column1 COMPACT 'MAJOR'")
-
-    checkAnswer(sql("select * from carbon_range_column1"), res)
-
-    deleteFile(filePath2)
-    deleteFile(filePath3)
-
-    sql("DROP TABLE IF EXISTS carbon_range_column1")
-  }
-
   test("Compact range_column with data skew") {
     sql("DROP TABLE IF EXISTS carbon_range_column4")
     sql(
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
index a70584b..6a83797 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
@@ -187,8 +187,6 @@
       (ID Int, date Timestamp, country String,
       name String, phonetype String, serialname String, salary Int)
       STORED BY 'org.apache.carbondata.format'
-      TBLPROPERTIES('DICTIONARY_EXCLUDE'='country,phonetype,serialname',
-      'DICTIONARY_INCLUDE'='ID')
       """)
 
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/emptyDimensionData.csv' into table carbon_table_1")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala
deleted file mode 100644
index b9b01f8..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.testsuite.datetype
-
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
-
-class DateTypeTest extends QueryTest with BeforeAndAfterAll{
-
-  override def afterAll: Unit = {
-    sql("DROP TABLE IF EXISTS Carbon_automation_testdate")
-    sql("DROP TABLE IF EXISTS Carbon_automation_testtimestamp")
-  }
-
-  test("must throw exception for date data type in dictionary_exclude") {
-    try {
-      sql(
-        "create table if not exists Carbon_automation_testdate (imei string,doj Date," +
-        "deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId " +
-        "string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string," +
-        "series string,productionDate timestamp,bomCode string,internalModels string, " +
-        "deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, " +
-        "deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict " +
-        "string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, " +
-        "ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, " +
-        "ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId " +
-        "string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber " +
-        "string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer " +
-        "string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, " +
-        "Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY" +
-        " int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province " +
-        "string, Latest_city string, Latest_district string, Latest_street string, " +
-        "Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, " +
-        "Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, " +
-        "Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, " +
-        "Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, " +
-        "Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber " +
-        "int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='doj')")
-
-      assert(false)
-    }
-    catch {
-      case exception: MalformedCarbonCommandException => assert(true)
-    }
-  }
-  test("must throw exception for timestamp data type in dictionary_exclude") {
-    sql(
-      "create table if not exists Carbon_automation_testtimestamp (imei string,doj timestamp," +
-      "deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId " +
-      "string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string," +
-      "series string,productionDate timestamp,bomCode string,internalModels string, " +
-      "deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, " +
-      "deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict " +
-      "string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, " +
-      "ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, " +
-      "ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId " +
-      "string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber " +
-      "string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer " +
-      "string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, " +
-      "Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY" +
-      " int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province " +
-      "string, Latest_city string, Latest_district string, Latest_street string, " +
-      "Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, " +
-      "Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, " +
-      "Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, " +
-      "Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, " +
-      "Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber " +
-      "int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='doj')")
-
-    assert(true)
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala
index 236dcfc..278cd92 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dblocation/DBLocationCarbonTableTestCase.scala
@@ -147,7 +147,7 @@
     sql("insert into carbontable select 'a',1,'aa','aaa'")
     sql("insert into carbontable select 'b',1,'bb','bbb'")
     sql("Alter table carbontable add columns(c4 string) " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='c4', 'DEFAULT.VALUE.c4'='def')")
+        "TBLPROPERTIES('DEFAULT.VALUE.c4'='def')")
     checkAnswer(
       sql("""select c1,c2,c3,c5,c4 from carbon.carbontable"""),
       Seq(Row("a",1,"aa","aaa","def"), Row("b",1,"bb","bbb","def"))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
index 811cb02..7ef17ce3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
@@ -105,8 +105,7 @@
     sql(
       "CREATE table CaseInsensitiveTable (ID int, date String, country String, name " +
       "String," +
-      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'" +
-      "TBLPROPERTIES('DICTIONARY_INCLUDE'='ID, salary')"
+      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'"
     )
     // table should drop wihout any error
     sql("drop table caseInsensitiveTable")
@@ -115,8 +114,7 @@
     sql(
       "CREATE table CaseInsensitiveTable (ID int, date String, country String, name " +
       "String," +
-      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'" +
-      "TBLPROPERTIES('DICTIONARY_INCLUDE'='ID, salary')"
+      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'"
     )
 
   }
@@ -126,8 +124,7 @@
     sql(
       "CREATE table default.table3 (ID int, date String, country String, name " +
       "String," +
-      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'" +
-      "TBLPROPERTIES('DICTIONARY_INCLUDE'='ID, salary')"
+      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'"
     )
     // table should drop without any error
     sql("drop table default.table3")
@@ -166,8 +163,7 @@
     sql(
       "CREATE table dropTableTest2 (ID int, date String, country String, name " +
       "String," +
-      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES('DICTIONARY_INCLUDE'='salary')"
+      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE dropTableTest2 " +
@@ -176,8 +172,7 @@
     sql(
       "CREATE table dropTableTest2 (ID int, date String, country String, name " +
       "String," +
-      "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES('DICTIONARY_INCLUDE'='date')"
+      "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE dropTableTest2 " +
@@ -192,8 +187,7 @@
       sql(
         "CREATE table test.dropTableTest3 (ID int, date String, country String, name " +
         "String," +
-        "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='salary')"
+        "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format' "
       )
       sql(
         s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE test.dropTableTest3 " +
@@ -202,8 +196,7 @@
       sql(
         "CREATE table test.dropTableTest3 (ID int, date String, country String, name " +
         "String," +
-        "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='date')"
+        "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata.format' "
       )
       sql(
         s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE test.dropTableTest3 " +
@@ -230,9 +223,7 @@
     sql(
       "CREATE table dropTableTest4 (ID int, date String, country String, name " +
       "String," +
-      "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata" +
-      ".format' " +
-      "TBLPROPERTIES('DICTIONARY_EXCLUDE'='date')"
+      "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/dataretention1.csv' INTO TABLE dropTableTest4 " +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/CastColumnTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/CastColumnTestCase.scala
index 24524b8..1c4939f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/CastColumnTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/CastColumnTestCase.scala
@@ -40,8 +40,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='empname,workgroupcategory, designation')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE DICTIONARY_CARBON_1 " +
@@ -68,8 +67,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='workgroupcategory', 'DICTIONARY_EXCLUDE'='empno, designation')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE NO_DICTIONARY_CARBON_2 " +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
index 05121dc..2589b13 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
@@ -53,8 +53,7 @@
       "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
       "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
       "projectenddate Timestamp, designation String,attendance Int,utilization " +
-      "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empname,designation')"
+      "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE NO_DICTIONARY_CARBON_6 " +
@@ -65,8 +64,7 @@
       "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
       "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
       "projectenddate Timestamp, designation String,attendance Int,utilization " +
-      "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')"
+      "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE NO_DICTIONARY_CARBON_7 " +
@@ -74,7 +72,7 @@
     )
     sql("CREATE TABLE filtertestTable (ID string,date Timestamp, country String, " +
       "name String, phonetype String, serialname String, salary Int) " +
-        "STORED BY 'org.apache.carbondata.format' " +  "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ID')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
         CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
index 7e74ac6..98a70a5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
@@ -31,12 +31,12 @@
     sql("DROP TABLE IF EXISTS hiveTable")
     sql("DROP TABLE IF EXISTS carbonEmpty")
     sql("DROP TABLE IF EXISTS hiveEmpty")
-    sql("CREATE TABLE carbonTable (imei String, age Int, num BigInt) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='age,num')")
+    sql("CREATE TABLE carbonTable (imei String, age Int, num BigInt) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/datawithNegtiveNumber.csv' INTO TABLE carbonTable")
     sql("CREATE TABLE hiveTable (imei String, age Int, num BigInt) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/datawithNegeativewithoutHeader.csv' INTO TABLE hiveTable")
 
-    sql("CREATE TABLE carbonEmpty (cust_id int, cust_name String, active_emui_version String, bob timestamp, bigint_column bigint) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='cust_name,active_emui_version')")
+    sql("CREATE TABLE carbonEmpty (cust_id int, cust_name String, active_emui_version String, bob timestamp, bigint_column bigint) STORED BY 'org.apache.carbondata.format' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/dataWithEmptyRows.csv' INTO TABLE carbonEmpty OPTIONS('FILEHEADER'='cust_id,cust_name,active_emui_version,bob,bigint_column')")
     sql("CREATE TABLE hiveEmpty (cust_id int, cust_name String, active_emui_version String, bob timestamp, bigint_column bigint) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/dataWithEmptyRows.csv' INTO TABLE hiveEmpty")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
index 87cc194..d5ac3d5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
@@ -55,8 +55,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='workgroupcategory')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE DICTIONARY_CARBON_6 " +
@@ -82,8 +81,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno, empname,designation','DICTIONARY_INCLUDE' = 'deptno')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE NO_DICTIONARY_CARBON_6 " +
@@ -108,8 +106,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno, empname,designation','DICTIONARY_INCLUDE' = 'deptno')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE NO_DICTIONARY_CARBON " +
@@ -134,8 +131,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno, empname,designation','DICTIONARY_INCLUDE' = 'deptno')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/rangedata.csv' INTO TABLE NO_DICTIONARY_CARBON_8 " +
@@ -162,8 +158,7 @@
         "ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>," +
         "proddate struct<productionDate:string,activeDeactivedate:array<string>>, gamePointId " +
         "double,contractNumber double) " +
-        "STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES ('DICTIONARY_INCLUDE'='deviceInformationId')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     //CarbonProperties.getInstance()
     //  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
@@ -265,7 +260,7 @@
 
   test("test range filter for less than filter"){
     sql("drop table if exists timestampTable")
-    sql("create table timestampTable (timestampCol timestamp) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='timestampCol')")
+    sql("create table timestampTable (timestampCol timestamp) stored by 'carbondata' ")
     sql(s"load data local inpath '$resourcesPath/timestamp.csv' into table timestampTable")
     checkAnswer(sql("select * from timestampTable where timestampCol='1970-01-01 05:30:00'"),
       sql("select * from timestampTable where timestampCol<='1970-01-01 05:30:00'"))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
index e9a83ec..9a99989 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
@@ -65,8 +65,7 @@
       "string, DOB " +
       "timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 " +
       "decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-      "double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-      "('DICTIONARY_EXCLUDE'='CUST_NAME')")
+      "double,INTEGER_COLUMN1 int) STORED BY 'org.apache.carbondata.format' ")
 
     sql(
       "create table NO_DICTIONARY_HIVE_6(empno string,empname string,designation string,doj " +
@@ -100,8 +99,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='workgroupcategory','DICTIONARY_EXCLUDE'='empno, empname,designation')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE NO_DICTIONARY_CARBON_6 " +
@@ -113,8 +111,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_INCLUDE'='workgroupcategory','DICTIONARY_EXCLUDE'='empname,designation')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE DICTIONARY_CARBON_6 " +
@@ -126,8 +123,7 @@
         "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
         "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
         "projectenddate Timestamp, designation String,attendance Int,utilization " +
-        "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')"
+        "Int,salary Int) STORED BY 'org.apache.carbondata.format' "
     )
     sql(
       s"LOAD DATA LOCAL INPATH '$resourcesPath/rangenodictionarycompare.csv' INTO TABLE NO_DICTIONARY_CARBON_7 " +
@@ -151,7 +147,7 @@
 
     sql("CREATE TABLE filtertestTable (ID string,date Timestamp, country String, " +
         "name String, phonetype String, serialname String, salary Int) " +
-        "STORED BY 'org.apache.carbondata.format' " +  "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ID')"
+        "STORED BY 'org.apache.carbondata.format' "
     )
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
index 195b107..4916733 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithNoDictTestCase.scala
@@ -43,7 +43,7 @@
         """
          CREATE TABLE IF NOT EXISTS directDictionaryTable
         (empno String, doj Date, salary Int)
-         STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='empno')"""
+         STORED BY 'org.apache.carbondata.format' """
       )
 
       CarbonProperties.getInstance()
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
index 6c8ab53..a6cde18 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
@@ -150,7 +150,7 @@
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
     sql(
       "create table test_timestamp(col timestamp) stored by 'carbondata' tblproperties" +
-      "('no_inverted_index'='col','dictionary_include'='col')")
+      "('no_inverted_index'='col')")
     val csvFilePath = s"$resourcesPath/data_timestamp.csv"
     sql(
       "load data inpath '" + csvFilePath +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryWithNoDictTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryWithNoDictTestCase.scala
index 4878c86..bbccd51 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryWithNoDictTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryWithNoDictTestCase.scala
@@ -46,7 +46,7 @@
         """
          CREATE TABLE IF NOT EXISTS directDictionaryTable
         (empno String, doj Timestamp, salary Int)
-         STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='empno')"""
+         STORED BY 'org.apache.carbondata.format'"""
       )
 
       CarbonProperties.getInstance()
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
index 861bbd4..70d90ca 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
@@ -395,7 +395,7 @@
 
   test("test if query is giving empty results for table with no segments") {
     sql("drop table if exists q1")
-    sql("create table q1(a string) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='a')")
+    sql("create table q1(a string) stored by 'carbondata' ")
     assert(sql("select * from q1 where a > 10").count() == 0)
     sql("drop table if exists q1")
   }
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestBetweenFilter.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestBetweenFilter.scala
index 0277883..4d6ee3d 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestBetweenFilter.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestBetweenFilter.scala
@@ -33,7 +33,7 @@
     sql("""
            CREATE TABLE IF NOT EXISTS carbonTableBetween
            (id String, name String, orders int)
-           STORED BY 'carbondata' TBLPROPERTIES ('DICTIONARY_INCLUDE'='id')
+           STORED BY 'carbondata'
         """)
     sql(s"""
            LOAD DATA LOCAL INPATH '$csvFilePath' into table carbonTableBetween OPTIONS('BAD_RECORDS_ACTION'='FORCE')
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestNotNullFilter.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestNotNullFilter.scala
index 2c29604..e31d3d1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestNotNullFilter.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/TestNotNullFilter.scala
@@ -37,7 +37,7 @@
            CREATE TABLE IF NOT EXISTS carbonTableNotNull
            (ID Int, date timestamp, country String,
            name String, phonetype String, serialname char(10), salary Int)
-           STORED BY 'carbondata' TBLPROPERTIES ('DICTIONARY_INCLUDE'='ID')
+           STORED BY 'carbondata'
            """)
      sql(s"""
            LOAD DATA LOCAL INPATH '$csvFilePath' into table carbonTableNotNull OPTIONS('BAD_RECORDS_ACTION'='FORCE')
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index fedfa70..cae86c0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -424,9 +424,7 @@
           "designation String, doj Timestamp, workgroupcategory int, " +
           "workgroupcategoryname String, deptno int, deptname String, projectcode int, " +
           "projectjoindate Timestamp, projectenddate Timestamp, attendance int, " +
-          "utilization int,salary int) STORED BY 'org.apache.carbondata.format' " +
-          "TBLPROPERTIES('DICTIONARY_INCLUDE'='empno,workgroupcategory,deptno,projectcode'," +
-          "'DICTIONARY_EXCLUDE'='empname')")
+          "utilization int,salary int) STORED BY 'org.apache.carbondata.format' ")
     }
   }
 
@@ -509,7 +507,6 @@
       .option("tableName", "study_carbondata")
       .option("compress", "true")  // just valid when tempCSV is true
       .option("tempCSV", "false")
-      .option("single_pass", "true")
       .option("sort_scope", "LOCAL_SORT")
       .mode(SaveMode.Append)
       .save()
@@ -554,24 +551,6 @@
     sql("DROP TABLE IF EXISTS iud.rand")
   }
 
-  test("Update operation on carbon table with singlepass") {
-    sql(s"""set ${ CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS }=true""")
-    sql("drop database if exists carbon1 cascade")
-    sql(s"create database carbon1 location '$dblocation'")
-    sql("use carbon1")
-    sql("""CREATE TABLE carbontable(id int, name string, city string, age int)
-         STORED BY 'org.apache.carbondata.format'""")
-    val testData = s"$resourcesPath/sample.csv"
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table carbontable")
-    // update operation
-    sql("""update carbon1.carbontable d  set (d.id) = (d.id + 1) where d.id > 2""").show()
-    checkAnswer(
-      sql("select count(*) from carbontable"),
-      Seq(Row(6))
-    )
-    sql(s"""set ${ CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS }=false""")
-    sql("drop table carbontable")
-  }
   test("Update operation on carbon table with persist false") {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE, "false")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/AllDataTypesTestCaseJoin.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/AllDataTypesTestCaseJoin.scala
index cd9b290..7f4bf00 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/AllDataTypesTestCaseJoin.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/AllDataTypesTestCaseJoin.scala
@@ -28,7 +28,7 @@
 class AllDataTypesTestCaseJoin extends QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {
-    sql("CREATE TABLE alldatatypestableJoin (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='empno','TABLE_BLOCKSIZE'='4')")
+    sql("CREATE TABLE alldatatypestableJoin (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('TABLE_BLOCKSIZE'='4')")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE alldatatypestableJoin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""");
 
     sql("CREATE TABLE alldatatypestableJoin_hive (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int)row format delimited fields terminated by ','")
@@ -61,9 +61,9 @@
     sql("DROP TABLE IF EXISTS carbon_table1")
     sql("DROP TABLE IF EXISTS carbon_table2")
 
-    sql("CREATE TABLE carbon_table1(shortField smallint,intField int,bigintField bigint,doubleField double,stringField string,timestampField timestamp,decimalField decimal(18,2),dateField date,charField char(5),floatField float) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='dateField, charField')")
+    sql("CREATE TABLE carbon_table1(shortField smallint,intField int,bigintField bigint,doubleField double,stringField string,timestampField timestamp,decimalField decimal(18,2),dateField date,charField char(5),floatField float) STORED BY 'carbondata' ")
 
-    sql("CREATE TABLE carbon_table2(shortField smallint,intField int,bigintField bigint,doubleField double,stringField string,timestampField timestamp,decimalField decimal(18,2),dateField date,charField char(5),floatField float) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='dateField, charField')")
+    sql("CREATE TABLE carbon_table2(shortField smallint,intField int,bigintField bigint,doubleField double,stringField string,timestampField timestamp,decimalField decimal(18,2),dateField date,charField char(5),floatField float) STORED BY 'carbondata' ")
 
     val path1 = s"$resourcesPath/join/data1.csv"
     val path2 = s"$resourcesPath/join/data2.csv"
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/JoinWithoutDictionaryColumn.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/JoinWithoutDictionaryColumn.scala
index 104e574..1568a73 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/JoinWithoutDictionaryColumn.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/joinquery/JoinWithoutDictionaryColumn.scala
@@ -41,20 +41,20 @@
 
     sql(
       """
-        create table mobile (mid String,Mobileid String, Color String, id int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Color')
+        create table mobile (mid String,Mobileid String, Color String, id int) STORED BY 'org.apache.carbondata.format'
       """)
     sql(
       """
-        create table emp (eid String,ename String, Mobileid String,Color String, id int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Color')
+        create table emp (eid String,ename String, Mobileid String,Color String, id int) STORED BY 'org.apache.carbondata.format'
       """)
 
     sql(
       """
-        create table mobile_d (mid String,Mobileid String, Color String, id int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Color','DICTIONARY_INCLUDE'='Mobileid')
+        create table mobile_d (mid String,Mobileid String, Color String, id int) STORED BY 'org.apache.carbondata.format'
       """)
     sql(
       """
-        create table emp_d (eid String,ename String, Mobileid String,Color String, id int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Color','DICTIONARY_INCLUDE'='Mobileid')
+        create table emp_d (eid String,ename String, Mobileid String,Color String, id int) STORED BY 'org.apache.carbondata.format'
       """)
 
     sql(
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportAlterTableTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportAlterTableTest.scala
index 431e3d4..3a237fc 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportAlterTableTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportAlterTableTest.scala
@@ -78,25 +78,6 @@
     }
   }
 
-  test("test alter table add column where same column is in dictionary include and local dictionary include") {
-    sql("drop table if exists local1")
-    sql(
-      """
-        | CREATE TABLE local1(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' tblproperties('local_dictionary_enable'='true',
-        | 'local_dictionary_threshold'='20000','local_dictionary_include'='city','no_inverted_index'='name')
-      """.stripMargin)
-    val exception = intercept[MalformedCarbonCommandException] {
-      sql(
-        "alter table local1 add columns (alt string) tblproperties('local_dictionary_include'='alt','dictionary_include'='alt')")
-    }
-    assert(exception.getMessage
-      .contains(
-        "LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: alt specified in Dictionary " +
-        "include. Local Dictionary will not be generated for Dictionary include columns. " +
-        "Please check the DDL."))
-  }
-
   test("test alter table add column where duplicate columns present in local dictionary include") {
     sql("drop table if exists local1")
     sql(
@@ -586,25 +567,6 @@
     assert(exception1.getMessage.contains("LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: id is not a string/complex/varchar datatype column. LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE should be no dictionary string/complex/varchar datatype column."))
   }
 
-  test("test alter set for local dictionary _010") {
-    sql("drop table if exists local1")
-    sql(
-      """
-        | CREATE TABLE local1(id int, name string, city string, age int)
-        | STORED BY 'carbondata' tblproperties('local_dictionary_enable'='true','dictionary_include'='name')
-      """.stripMargin)
-
-    val descLoc1 = sql("describe formatted local1").collect
-    descLoc1.find(_.get(0).toString.contains("Local Dictionary Enabled")) match {
-      case Some(row) => assert(row.get(1).toString.contains("true"))
-      case None => assert(false)
-    }
-    val exception1 = intercept[Exception] {
-      sql("alter table local1 set tblproperties('local_dictionary_include'='name')")
-    }
-    assert(exception1.getMessage.contains("LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: name specified in Dictionary include. Local Dictionary will not be generated for Dictionary include columns. Please check the DDL."))
-  }
-
   test("test alter set for local dictionary _011") {
     sql("drop table if exists local1")
     sql(
@@ -653,25 +615,6 @@
     assert(exception1.getMessage.contains("LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column:  does not exist in table. Please check the DDL."))
   }
 
-  test("test alter set for local dictionary _013") {
-    sql("drop table if exists local1")
-    sql(
-      """
-        | CREATE TABLE local1(id int, name string, city string, age int)
-        | STORED BY 'carbondata' tblproperties('local_dictionary_enable'='true','dictionary_include'='name')
-      """.stripMargin)
-
-    val descLoc1 = sql("describe formatted local1").collect
-    descLoc1.find(_.get(0).toString.contains("Local Dictionary Enabled")) match {
-      case Some(row) => assert(row.get(1).toString.contains("true"))
-      case None => assert(false)
-    }
-    val exception1 = intercept[Exception] {
-      sql("alter table local1 set tblproperties('local_dictionary_exclude'='name')")
-    }
-    assert(exception1.getMessage.contains("LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: name specified in Dictionary include. Local Dictionary will not be generated for Dictionary include columns. Please check the DDL."))
-  }
-
   test("test alter set for local dictionary _014") {
     sql("drop table if exists local1")
     sql(
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportCreateTableTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportCreateTableTest.scala
index ee67c50..d4738a0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportCreateTableTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportCreateTableTest.scala
@@ -132,19 +132,6 @@
         "be no dictionary string/complex/varchar datatype column"))
   }
 
-  test("test local dictionary custom configurations for local dict columns _006") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name','local_dictionary_include'='name')
-        """.
-          stripMargin)
-    }
-  }
-
   test("test local dictionary custom configurations for local dict threshold _001") {
     sql("drop table if exists local1")
     sql(
@@ -370,20 +357,6 @@
   }
 
   test("test local dictionary custom configurations with both columns and threshold configured " +
-       "_008") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='20000','local_dictionary_include'='name',
-          | 'dictionary_include'='name')
-        """.stripMargin)
-    }
-  }
-
-  test("test local dictionary custom configurations with both columns and threshold configured " +
        "_009") {
     sql("drop table if exists local1")
     intercept[MalformedCarbonCommandException] {
@@ -409,20 +382,6 @@
     }
   }
 
-  test("test local dictionary custom configurations with both columns and threshold configured " +
-       "_011") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='23213497321591234324',
-          | 'local_dictionary_include'='name','dictionary_include'='name')
-        """.stripMargin)
-    }
-  }
-
   test("test local dictionary default configuration when enabled") {
     sql("drop table if exists local1")
     sql(
@@ -531,20 +490,6 @@
         "be no dictionary string/complex/varchar datatype column"))
   }
 
-  test("test local dictionary custom configurations when enabled for local dict columns _006") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
-          | 'local_dictionary_include'='name')
-        """.
-          stripMargin)
-    }
-  }
-
   test("test local dictionary custom configurations when local_dictionary_exclude is " +
        "configured _001") {
     sql("drop table if exists local1")
@@ -640,21 +585,6 @@
         "be no dictionary string/complex/varchar datatype column"))
   }
 
-  test("test local dictionary custom configurations when local_dictionary_exclude is " +
-       "configured _006") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
-          | 'local_dictionary_exclude'='name')
-        """.
-          stripMargin)
-    }
-  }
-
   test("test local dictionary custom configurations when local_dictionary_include and " +
        "local_dictionary_exclude is configured _001") {
     sql("drop table if exists local1")
@@ -734,21 +664,6 @@
   }
 
   test("test local dictionary custom configurations when local_dictionary_include and " +
-       "local_dictionary_exclude is configured _004") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_exclude'='name','local_dictionary_include'='city',
-          | 'local_dictionary_enable'='true','dictionary_include'='name,city')
-        """.
-          stripMargin)
-    }
-  }
-
-  test("test local dictionary custom configurations when local_dictionary_include and " +
        "local_dictionary_exclude is configured _005") {
     sql("drop table if exists local1")
     val exception = intercept[MalformedCarbonCommandException] {
@@ -1044,20 +959,6 @@
   }
 
   test("test local dictionary custom configurations when enabled with both columns and threshold " +
-       "configured _008") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','local_dictionary_threshold'='20000',
-          | 'local_dictionary_include'='name','dictionary_include'='name')
-        """.stripMargin)
-    }
-  }
-
-  test("test local dictionary custom configurations when enabled with both columns and threshold " +
        "configured _009") {
     sql("drop table if exists local1")
     intercept[MalformedCarbonCommandException] {
@@ -1085,21 +986,6 @@
     }
   }
 
-  test("test local dictionary custom configurations when enabled with both columns and threshold " +
-       "configured _011") {
-    sql("drop table if exists local1")
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1(id int, name string, city string, age int)
-          | STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true',
-          | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name',
-          | 'dictionary_include'='name')
-        """.stripMargin)
-    }
-  }
-
   test("test local dictionary default configuration when disabled") {
     sql("drop table if exists local1")
     sql(
@@ -1202,7 +1088,7 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('local_dictionary_enable'='false','dictionary_include'='name',
+        | tblproperties('local_dictionary_enable'='false',
         | 'local_dictionary_include'='name')
       """.
         stripMargin)
@@ -1429,7 +1315,7 @@
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
         | tblproperties('local_dictionary_enable'='false','local_dictionary_threshold'='20000',
-        | 'local_dictionary_include'='name','dictionary_include'='name')
+        | 'local_dictionary_include'='name')
       """.stripMargin)
 
     val descLoc = sql("describe formatted local1").collect
@@ -1483,8 +1369,7 @@
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
         | tblproperties('local_dictionary_enable'='false',
-        | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name',
-        | 'dictionary_include'='name')
+        | 'local_dictionary_threshold'='23213497321591234324','local_dictionary_include'='name')
       """.stripMargin)
 
     val descLoc = sql("describe formatted local1").collect
@@ -1501,7 +1386,7 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('local_dictionary_enable'='true','dictionary_include'='city','sort_scope'='global_sort',
+        | tblproperties('local_dictionary_enable'='true','sort_scope'='global_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1523,7 +1408,8 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='local_sort',
+        | tblproperties(
+        | 'sort_scope'='local_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1544,7 +1430,8 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='no_sort',
+        | tblproperties(
+        | 'sort_scope'='no_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1565,7 +1452,8 @@
       """
         | CREATE TABLE local1(id int, name string, city string, age int)
         | STORED BY 'org.apache.carbondata.format'
-        | tblproperties('dictionary_include'='city','sort_scope'='local_sort',
+        | tblproperties(
+        | 'sort_scope'='local_sort',
         | 'sort_columns'='city,name')
       """.stripMargin)
 
@@ -1733,26 +1621,6 @@
         "be no dictionary string/complex/varchar datatype column"))
   }
 
-  test("test CTAS statements for local dictionary custom configurations when enabled for local dict " +
-       "columns _006") {
-    sql("drop table if exists local")
-    sql("drop table if exists local1")
-    sql(
-      """
-        | CREATE TABLE local(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' tblproperties('local_dictionary_enable'='false')
-      """.stripMargin)
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1 STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
-          | 'local_dictionary_include'='name') as select * from local
-        """.
-          stripMargin)
-    }
-  }
-
   test("test CTAS statements for local dictionary custom configurations when " +
        "local_dictionary_exclude is configured _001") {
     sql("drop table if exists local")
@@ -1874,26 +1742,6 @@
   }
 
   test("test CTAS statements for local dictionary custom configurations when " +
-       "local_dictionary_exclude is configured _006") {
-    sql("drop table if exists local1")
-    sql("drop table if exists local")
-    sql(
-      """
-        | CREATE TABLE local(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' tblproperties('local_dictionary_enable'='false')
-      """.stripMargin)
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1 STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_enable'='true','dictionary_include'='name',
-          | 'local_dictionary_exclude'='name') as select * from local
-        """.
-          stripMargin)
-    }
-  }
-
-  test("test CTAS statements for local dictionary custom configurations when " +
        "local_dictionary_include and local_dictionary_exclude is configured _001") {
     sql("drop table if exists local1")
     sql("drop table if exists local")
@@ -1953,27 +1801,6 @@
   }
 
   test("test CTAS statements for local dictionary custom configurations when " +
-       "local_dictionary_include and local_dictionary_exclude is configured _003") {
-    sql("drop table if exists local1")
-    sql("drop table if exists local")
-    sql(
-      """
-        | CREATE TABLE local(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format' tblproperties('local_dictionary_enable'='false')
-      """.stripMargin)
-    intercept[MalformedCarbonCommandException] {
-      sql(
-        """
-          | CREATE TABLE local1 STORED BY 'org.apache.carbondata.format'
-          | tblproperties('local_dictionary_exclude'='name','local_dictionary_include'='city',
-          | 'local_dictionary_enable'='true','dictionary_include'='name,city') as select * from
-          | local
-        """.
-          stripMargin)
-    }
-  }
-
-  test("test CTAS statements for local dictionary custom configurations when " +
        "local_dictionary_include and local_dictionary_exclude is configured _004") {
     sql("drop table if exists local1")
     sql("drop table if exists local")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
index adcbcd5..eb88c36 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportLoadTableTest.scala
@@ -78,20 +78,19 @@
     assert(checkForLocalDictionary(getDimRawChunk(0)))
   }
 
-  test("test local dictionary generation for local dictioanry include") {
+  test("test local dictionary generation for local dictionary include") {
     sql("drop table if exists local2")
-    sql(
-      "CREATE TABLE local2(name string) STORED BY 'carbondata' tblproperties" +
-      "('dictionary_include'='name')")
+    sql("CREATE TABLE local2(name string) STORED BY 'carbondata' " +
+        "TBLPROPERTIES ('local_dictionary_enable'='true', 'local_dictionary_include'='name')")
     sql("load data inpath '" + file1 + "' into table local2 OPTIONS('header'='false')")
-    assert(!checkForLocalDictionary(getDimRawChunk(0)))
+    assert(checkForLocalDictionary(getDimRawChunk(0)))
   }
 
   test("test local dictionary generation for local dictioanry exclude"){
     sql("drop table if exists local2")
     sql(
       "CREATE TABLE local2(name string) STORED BY 'carbondata' tblproperties" +
-      "('local_dictionary_enable'='true','dictionary_exclude'='name')")
+      "('local_dictionary_enable'='true')")
     sql("load data inpath '" + file1 + "' into table local2 OPTIONS('header'='false')")
     assert(checkForLocalDictionary(getDimRawChunk(0)))
   }
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
index 428a56a..6c8f7a0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
@@ -82,34 +82,6 @@
     sql(s"drop table if exists $longStringTable")
   }
 
-  test("long string columns cannot be dictionary include") {
-    val exceptionCaught = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | CREATE TABLE if not exists $longStringTable(
-           | id INT, name STRING, description STRING, address STRING, note STRING
-           | ) STORED BY 'carbondata'
-           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note', 'dictionary_include'='address')
-           |""".
-          stripMargin)
-    }
-    assert(exceptionCaught.getMessage.contains("DICTIONARY_INCLUDE is unsupported for long string datatype column: address"))
-  }
-
-  test("long string columns cannot be dictionay exclude") {
-    val exceptionCaught = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-           | CREATE TABLE if not exists $longStringTable(
-           | id INT, name STRING, description STRING, address STRING, note STRING
-           | ) STORED BY 'carbondata'
-           | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note', 'dictionary_exclude'='address')
-           |""".
-          stripMargin)
-    }
-    assert(exceptionCaught.getMessage.contains("DICTIONARY_EXCLUDE is unsupported for long string datatype column: address"))
-  }
-
   test("long string columns cannot be sort_columns") {
     val exceptionCaught = intercept[MalformedCarbonCommandException] {
       sql(
@@ -460,7 +432,6 @@
     longStringDF.write
       .format("carbondata")
       .option("tableName", longStringTable)
-      .option("single_pass", "false")
       .option("sort_columns", "name")
       .option("long_string_columns", "description, note")
       .mode(SaveMode.Overwrite)
@@ -475,7 +446,7 @@
          | CREATE TABLE if not exists $longStringTable(
          | id INT, name STRING, description STRING, address STRING, note STRING
          | ) STORED BY 'carbondata'
-         | TBLPROPERTIES('LONG_STRING_COLUMNS'='description, note', 'dictionary_include'='name', 'sort_columns'='id')
+         | TBLPROPERTIES('LONG_STRING_COLUMNS'='description, note', 'sort_columns'='id')
          |""".
         stripMargin)
 
@@ -497,7 +468,7 @@
          | CREATE TABLE if not exists $longStringTable(
          | id INT, name STRING, description STRING, address STRING, note STRING
          | ) STORED BY 'carbondata'
-         | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note', 'dictionary_include'='name', 'sort_columns'='id')
+         | TBLPROPERTIES('LONG_STRING_COLUMNS'='address, note', 'sort_columns'='id')
          |""".
         stripMargin)
     checkExistence(sql(s"desc formatted $longStringTable"), true, "long_string_columns".toUpperCase, "address", "note")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala
deleted file mode 100644
index e0a41b3..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/predefdic/TestPreDefDictionary.scala
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.testsuite.predefdic
-
-import org.apache.spark.sql.Row
-import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.util.QueryTest
-
-/**
- * Test cases for testing columns having \N or \null values for non numeric columns
- */
-class TestPreDefDictionary extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    sql("DROP TABLE IF EXISTS predefdictable")
-    sql("DROP TABLE IF EXISTS predefdictable1")
-    sql("DROP TABLE IF EXISTS columndicTable")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT
-      )
-  }
-
-  test("dictionary value not present in the allpredefdictionary dic file must be loaded.") {
-    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
-    val testData = s"$resourcesPath/predefdic/data3.csv"
-    val csvHeader = "ID,phonetype"
-    val allDictFile = s"$resourcesPath/predefdic/allpredefdictionary.csv"
-    sql(
-      """CREATE TABLE IF NOT EXISTS predefdictable (ID Int, phonetype String)
-       STORED BY 'carbondata'""")
-    sql(
-      s"""LOAD DATA LOCAL INPATH '$testData' into table predefdictable
-           options('ALL_DICTIONARY_PATH'='$allDictFile','single_pass'='true')""")
-    checkAnswer(
-      sql("select phonetype from predefdictable where phonetype='phone197'"),
-      Seq(Row("phone197"))
-    )
-  }
-
-  test("dictionary value not present in the allpredefdictionary dic with single_pass.") {
-    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
-    val testData = s"$resourcesPath/predefdic/data3.csv"
-    val csvHeader = "ID,phonetype"
-    val allDictFile = s"$resourcesPath/predefdic/allpredefdictionary.csv"
-    sql(
-      """CREATE TABLE IF NOT EXISTS predefdictable1 (ID Int, phonetype String)
-       STORED BY 'carbondata'""")
-    sql(
-      s"""LOAD DATA LOCAL INPATH '$testData' into table predefdictable1
-           options('ALL_DICTIONARY_PATH'='$allDictFile', 'SINGLE_PASS'='true')""")
-    checkAnswer(
-      sql("select phonetype from predefdictable1 where phonetype='phone197'"),
-      Seq(Row("phone197"))
-    )
-  }
-
-  test("dictionary value not present in the columndict dic with single_pass.") {
-    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
-    val testData = s"$resourcesPath/predefdic/data3.csv"
-    val csvHeader = "ID,phonetype"
-    val dicFilePath = s"$resourcesPath/predefdic/dicfilepath.csv"
-    sql(
-      """CREATE TABLE IF NOT EXISTS columndicTable (ID Int, phonetype String)
-       STORED BY 'carbondata'""")
-    sql(
-      s"""LOAD DATA LOCAL INPATH '$testData' into table columndicTable
-           options('COLUMNDICT'='phonetype:$dicFilePath', 'SINGLE_PASS'='true')""")
-    checkAnswer(
-      sql("select phonetype from columndicTable where phonetype='phone197'"),
-      Seq(Row("phone197"))
-    )
-  }
-
-  test("validation test columndict with single_pass= false.") {
-    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
-    val testData = s"$resourcesPath/predefdic/data3.csv"
-    val csvHeader = "ID,phonetype"
-    val dicFilePath = s"$resourcesPath/predefdic/dicfilepath.csv"
-    sql(
-      """CREATE TABLE IF NOT EXISTS columndicValidationTable (ID Int, phonetype String)
-       STORED BY 'carbondata'""")
-    try {
-      sql(
-        s"""LOAD DATA LOCAL INPATH '$testData' into table columndicValidationTable
-           options('COLUMNDICT'='phonetype:$dicFilePath', 'SINGLE_PASS'='false')""")
-    } catch {
-      case x: Throwable =>
-        val failMess: String = "Can not use all_dictionary_path or columndict without single_pass."
-        assert(failMess.equals(x.getMessage))
-    }
-  }
-
-  test("validation test ALL_DICTIONARY_PATH with single_pass= false.") {
-    val csvFilePath = s"$resourcesPath/nullvalueserialization.csv"
-    val testData = s"$resourcesPath/predefdic/data3.csv"
-    val csvHeader = "ID,phonetype"
-    val allDictFile = s"$resourcesPath/predefdic/allpredefdictionary.csv"
-    sql(
-      """CREATE TABLE IF NOT EXISTS predefdictableval (ID Int, phonetype String)
-       STORED BY 'carbondata'""")
-    try {
-    sql(
-      s"""LOAD DATA LOCAL INPATH '$testData' into table predefdictableval
-           options('ALL_DICTIONARY_PATH'='$allDictFile', 'SINGLE_PASS'='false')""")
-    } catch {
-      case x: Throwable =>
-        val failMess: String = "Can not use all_dictionary_path or columndict without single_pass."
-        assert(failMess.equals(x.getMessage))
-    }
-  }
-
-  override def afterAll {
-    sql("DROP TABLE IF EXISTS predefdictable")
-    sql("DROP TABLE IF EXISTS predefdictable1")
-    sql("DROP TABLE IF EXISTS columndicTable")
-    sql("DROP TABLE IF EXISTS columndicValidationTable")
-    sql("DROP TABLE IF EXISTS predefdictableval")
-
-  }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index a6a48d5..5c889ce 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -50,7 +50,7 @@
       "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-      "('dictionary_include' = 'empno', 'sort_columns'='empno')")
+      "('sort_columns'='empno')")
   }
 
   test("create table sort columns dictionary exclude - int") {
@@ -59,7 +59,7 @@
       "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-      "('dictionary_exclude' = 'empno', 'sort_columns'='empno')")
+      "('sort_columns'='empno')")
   }
 
   test("create table sort columns dictionary include - bigint") {
@@ -68,7 +68,7 @@
       "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-      "('dictionary_include' = 'empno', 'sort_columns'='empno')")
+      "('sort_columns'='empno')")
   }
 
   test("create table sort columns dictionary exclude - bigint") {
@@ -77,7 +77,7 @@
       "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-      "('dictionary_exclude' = 'empno', 'sort_columns'='empno')")
+      "( 'sort_columns'='empno')")
   }
 
   test("create table with no dictionary sort_columns") {
@@ -92,7 +92,7 @@
       " workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties" +
-      "('dictionary_exclude'='empno','sort_columns'='empno', 'SORT_SCOPE'='local_sort')")
+      "('sort_columns'='empno', 'SORT_SCOPE'='local_sort')")
     sql(
       s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable1a OPTIONS
          |('DELIMITER'= ',', 'QUOTECHAR'= '\"')""".stripMargin)
@@ -110,7 +110,7 @@
       " workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties" +
-      "('dictionary_exclude'='empno,empname,workgroupcategoryname','sort_columns'='empno,empname'," +
+      "('sort_columns'='empno,empname'," +
       "'SORT_SCOPE'='local_sort')")
     sql(
       s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable1b OPTIONS
@@ -120,7 +120,7 @@
   }
 
   test("create table with dictionary sort_columns") {
-    sql("CREATE TABLE sorttable2 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname', 'dictionary_include'='empname')")
+    sql("CREATE TABLE sorttable2 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname')")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable2 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
     checkAnswer(sql("select empname from sorttable2"),sql("select empname from origintable1"))
   }
@@ -200,7 +200,7 @@
   }
 
   test("filter on sort_columns include no-dictionary, direct-dictionary and dictioanry") {
-    sql("CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, doj, empname', 'dictionary_include'='empname')")
+    sql("CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, doj, empname')")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
     // no dictionary
     checkAnswer(sql("select * from sorttable6 where workgroupcategory = 1"), sql("select * from origintable1 where workgroupcategory = 1 order by doj"))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
index ec05108..239e53b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumnsWithUnsafe.scala
@@ -39,7 +39,7 @@
   }
 
   test("create table with dictionary sort_columns") {
-    sql("CREATE TABLE sorttable2 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname', 'dictionary_include'='empname')")
+    sql("CREATE TABLE sorttable2 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='empname')")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable2 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')""")
     checkAnswer(sql("select empname from sorttable2"),sql("select empname from origintable1"))
   }
@@ -145,7 +145,7 @@
   }
 
   test("filter on sort_columns include no-dictionary, direct-dictionary and dictioanry") {
-    sql("CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, doj, empname', 'dictionary_include'='empname')")
+    sql("CREATE TABLE sorttable6 (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('sort_columns'='workgroupcategory, doj, empname')")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE sorttable6 OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'TIMESTAMPFORMAT'='dd-MM-yyyy')""")
     // no dictionary
     checkAnswer(sql("select * from sorttable6 where workgroupcategory = 1"), sql("select * from origintable1 where workgroupcategory = 1 order by doj"))
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
index f89f84d..3f9e994 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
@@ -95,20 +95,6 @@
 
   }
 
-  test("single pass loading for global sort partition table for one partition column") {
-    sql(
-      """
-        | CREATE TABLE singlepasspartitionone (empname String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (designation String)
-        | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE singlepasspartitionone OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='true')""")
-
-  }
-
   test("data loading for global sort partition table for one static partition column with load syntax") {
     sql(
       """
@@ -643,7 +629,7 @@
         CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, LoggerAction.FAIL.name())
       sql("drop table if exists partdatecarb2")
       sql(
-        "create table partdatecarb2(name string, dob string) partitioned by(age Int) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='age')")
+        "create table partdatecarb2(name string, dob string) partitioned by(age Int) stored by 'carbondata' ")
 
       sql("insert into partdatecarb2 partition(age='12') select 'name1','2016-06-28'")
       checkAnswer(sql("select name,age,cast(dob as string) from partdatecarb2"),
@@ -654,14 +640,6 @@
     }
   }
 
-  test("partition with int issue and dictionary include") {
-    sql("drop table if exists partdatecarb3")
-    intercept[Exception] {
-      sql(
-        "create table partdatecarb3(name string, dob string) partitioned by(age Int) stored by 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='age')")
-    }
-  }
-
   test("data loading for all dimensions with table for two partition column") {
     sql("drop table if exists partitiontwoalldims")
     sql(
@@ -760,7 +738,7 @@
   test("partition colunm test without partition column in fileheader of load command") {
     sql("DROP TABLE IF EXISTS partitiontablewithoutpartcolumninfileheader")
 
-    sql("CREATE TABLE partitiontablewithoutpartcolumninfileheader (CUST_ID int,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) partitioned by(CUST_NAME String) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='CUST_ID,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1') ")
+    sql("CREATE TABLE partitiontablewithoutpartcolumninfileheader (CUST_ID int,ACTIVE_EMUI_VERSION string, DOB timestamp, DOJ timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) partitioned by(CUST_NAME String) STORED BY 'org.apache.carbondata.format' ")
     sql(s"""LOAD DATA INPATH '$resourcesPath/data_with_all_types.csv' into table partitiontablewithoutpartcolumninfileheader partition(cust_name='ravi') OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME1,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""")
 
     checkAnswer(sql("select count(*) from partitiontablewithoutpartcolumninfileheader"), Seq(Row(10)))
@@ -1049,7 +1027,6 @@
     sql("drop table if exists partitionmultiplethree")
     sql("drop table if exists insertpartitionthree")
     sql("drop table if exists staticpartitionone")
-    sql("drop table if exists singlepasspartitionone")
     sql("drop table if exists loadstaticpartitionone")
     sql("drop table if exists loadstaticpartitiononeoverwrite")
     sql("drop table if exists streamingpartitionedtable")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
index d3cc9ac..14fa961 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
@@ -166,7 +166,6 @@
         |  utilization int,salary int)
         | PARTITIONED BY (workgroupcategory int, empname String, designation String)
         | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('DICTIONARY_INCLUDE'='deptname')
       """.stripMargin)
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE partitionmultiplethree OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE partitionmultiplethree OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
@@ -213,21 +212,6 @@
     validateDataFiles("default_staticpartitionone", "0", 1)
   }
 
-  test("single pass loading for partition table for one partition column") {
-    sql(
-      """
-        | CREATE TABLE singlepasspartitionone (empname String, doj Timestamp,
-        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
-        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
-        |  utilization int,salary int)
-        | PARTITIONED BY (designation String)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE singlepasspartitionone OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='true')""")
-
-    validateDataFiles("default_singlepasspartitionone", "0", 8)
-  }
-
   test("data loading for partition table for one static partition column with load syntax") {
     sql(
       """
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala
index 9f2de4b..556df30 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonDropCacheCommand.scala
@@ -51,8 +51,7 @@
     sql(s"CREATE TABLE $tableName(empno int, empname String, designation String, " +
         s"doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, " +
         s"deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp," +
-        s"attendance int,utilization int, salary int) stored by 'carbondata' " +
-        s"TBLPROPERTIES('DICTIONARY_INCLUDE'='designation, workgroupcategoryname')")
+        s"attendance int,utilization int, salary int) stored by 'carbondata' ")
     sql(s"LOAD DATA INPATH '$resourcesPath/data.csv' INTO TABLE $tableName")
     sql(s"LOAD DATA INPATH '$resourcesPath/data.csv' INTO TABLE $tableName")
     sql(s"SELECT * FROM $tableName").collect()
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala
index a219bfb..aa9bf52 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/TestCarbonShowCacheCommand.scala
@@ -42,7 +42,6 @@
         |  projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,
         |  salary int)
         | STORED BY 'org.apache.carbondata.format'
-        | TBLPROPERTIES('DICTIONARY_INCLUDE'='deptname')
       """.stripMargin)
     // bloom
     sql("CREATE DATAMAP IF NOT EXISTS cache_1_bloom ON TABLE cache_db.cache_1 USING 'bloomfilter' " +
diff --git a/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
index 1ec6aec..d2689b9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/spark/sql/profiler/ProfilerSuite.scala
@@ -120,8 +120,8 @@
     checkCommand("DROP TABLE IF EXISTS mobile")
     checkCommand("DROP TABLE IF EXISTS emp")
     // create table
-    checkCommand("CREATE TABLE mobile (mid string,mobileId string, color string, id int) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Color')")
-    checkCommand("CREATE TABLE emp (eid string,eName string, mobileId string,color string, id int) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Color')")
+    checkCommand("CREATE TABLE mobile (mid string,mobileId string, color string, id int) STORED BY 'carbondata' ")
+    checkCommand("CREATE TABLE emp (eid string,eName string, mobileId string,color string, id int) STORED BY 'carbondata'")
     // load data
     checkCommand(s"LOAD DATA LOCAL INPATH '$resourcesPath/join/mobile.csv' INTO TABLE mobile OPTIONS('FILEHEADER'='mid,mobileId,color,id')")
     checkCommand(s"LOAD DATA LOCAL INPATH '$resourcesPath/join/employee.csv' INTO TABLE emp OPTIONS('FILEHEADER'='eid,eName,mobileId,color,id')")
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClient.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClient.java
deleted file mode 100644
index b5f9da2..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClient.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.dictionary.client;
-
-import java.nio.charset.Charset;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-
-import com.google.common.collect.Lists;
-import io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.log4j.Logger;
-import org.apache.spark.SecurityManager;
-import org.apache.spark.SparkConf;
-import org.apache.spark.network.TransportContext;
-import org.apache.spark.network.client.TransportClient;
-import org.apache.spark.network.client.TransportClientBootstrap;
-import org.apache.spark.network.client.TransportClientFactory;
-import org.apache.spark.network.netty.SparkTransportConf;
-import org.apache.spark.network.sasl.SaslClientBootstrap;
-import org.apache.spark.network.util.TransportConf;
-
-/**
- * Dictionary client to connect to Dictionary server and generate dictionary values
- */
-public class SecureDictionaryClient implements DictionaryClient {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(SecureDictionaryClient.class.getName());
-
-  private SecureDictionaryClientHandler dictionaryClientHandler =
-      new SecureDictionaryClientHandler();
-
-  private NioEventLoopGroup workerGroup;
-  private TransportClient client;
-  private TransportClientFactory clientFactory;
-
-  /**
-   * start dictionary client
-   *
-   * @param address
-   * @param port
-   */
-  @Override
-  public void startClient(String secretKey, String address, int port,
-      boolean encryptSecureServer) {
-    LOGGER.info("Starting client on " + address + " " + port);
-    long start = System.currentTimeMillis();
-
-    SecurityManager securityMgr;
-    SparkConf conf = new SparkConf().setAppName("Carbon Dictionary Client");
-
-    conf.set("spark.authenticate", "true");
-
-    if (null != secretKey) {
-      conf.set("spark.authenticate.secret", secretKey);
-    }
-
-    if (encryptSecureServer) {
-      conf.set("spark.authenticate.enableSaslEncryption", "true");
-    }
-
-    TransportConf transportConf =
-        SparkTransportConf.fromSparkConf(conf, "Carbon Dictionary Client", 0);
-    if (null != secretKey) {
-      securityMgr = new SecurityManager(conf, scala.Option.apply(secretKey.getBytes(Charset.forName(
-          CarbonCommonConstants.DEFAULT_CHARSET))));
-    } else {
-      securityMgr = new SecurityManager(conf, null);
-    }
-
-    TransportContext context = new TransportContext(transportConf, dictionaryClientHandler);
-    clientFactory = context.createClientFactory(Lists.<TransportClientBootstrap>newArrayList(
-        new SaslClientBootstrap(transportConf, "Carbon Dictionary Client", securityMgr)));
-
-    try {
-      client = clientFactory.createClient(address, port);
-    } catch (Exception e) {
-      LOGGER.error("Dictionary Client Failed to bind to port:", e);
-    }
-    LOGGER.info(
-        "Dictionary client Started, Total time spent : " + (System.currentTimeMillis() - start));
-  }
-
-  /**
-   * for client request
-   *
-   * @param key
-   * @return
-   */
-  @Override
-  public DictionaryMessage getDictionary(DictionaryMessage key) {
-    return dictionaryClientHandler.getDictionary(key, this.client);
-  }
-
-  /**
-   * shutdown dictionary client
-   */
-  @Override
-  public void shutDown() {
-    clientFactory.close();
-  }
-}
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java
deleted file mode 100644
index 2456b98..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/client/SecureDictionaryClientHandler.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.dictionary.client;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.Unpooled;
-import org.apache.log4j.Logger;
-import org.apache.spark.network.client.RpcResponseCallback;
-import org.apache.spark.network.client.TransportClient;
-import org.apache.spark.network.server.OneForOneStreamManager;
-import org.apache.spark.network.server.RpcHandler;
-import org.apache.spark.network.server.StreamManager;
-
-/**
- * Client handler to get data.
- */
-public class SecureDictionaryClientHandler extends RpcHandler {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(SecureDictionaryClientHandler.class.getName());
-
-  private final BlockingQueue<DictionaryMessage> responseMsgQueue = new LinkedBlockingQueue<>();
-
-  /**
-   * client send request to server
-   *
-   * @param key DictionaryMessage
-   * @return DictionaryMessage
-   */
-  public DictionaryMessage getDictionary(DictionaryMessage key, TransportClient client) {
-    DictionaryMessage dictionaryMessage;
-    ByteBuffer resp = null;
-    try {
-
-      ByteBuf buffer = ByteBufAllocator.DEFAULT.heapBuffer();
-      key.writeData(buffer);
-      resp = client.sendRpcSync(buffer.nioBuffer(), 100000);
-    } catch (Exception e) {
-      LOGGER.error("Error while send request to server ", e);
-    }
-    try {
-      if (resp == null) {
-        StringBuilder message = new StringBuilder();
-        message.append("DictionaryMessage { ColumnName: ").append(key.getColumnName())
-            .append(", DictionaryValue: ").append(key.getDictionaryValue()).append(", type: ")
-            .append(key.getType()).append(" }");
-        throw new RuntimeException("Request timed out for key : " + message);
-      }
-      DictionaryMessage newKey = new DictionaryMessage();
-      ByteBuf data = Unpooled.wrappedBuffer(resp);
-      newKey.readFullLength(data);
-      data.release();
-      return newKey;
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public void receive(TransportClient transportClient, ByteBuffer byteBuffer,
-      RpcResponseCallback rpcResponseCallback) {
-    try {
-      ByteBuf data = Unpooled.wrappedBuffer(byteBuffer);
-      DictionaryMessage key = new DictionaryMessage();
-      key.readFullLength(data);
-      data.release();
-      if (responseMsgQueue.offer(key)) {
-        LOGGER.info("key: " + key + " added to queue");
-      } else {
-        LOGGER.error("Failed to add key: " + key + " to queue");
-      }
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-      throw e;
-    }
-  }
-
-  @Override
-  public StreamManager getStreamManager() {
-    return new OneForOneStreamManager();
-  }
-
-}
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/provider/SecureDictionaryServiceProvider.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/provider/SecureDictionaryServiceProvider.java
deleted file mode 100644
index eb60b7f..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/provider/SecureDictionaryServiceProvider.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.dictionary.provider;
-
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.service.DictionaryServiceProvider;
-import org.apache.carbondata.spark.dictionary.client.SecureDictionaryClient;
-
-public class SecureDictionaryServiceProvider implements DictionaryServiceProvider {
-
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  @Override
-  public DictionaryClient getDictionaryClient() {
-    return new SecureDictionaryClient();
-  }
-}
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java
deleted file mode 100644
index 2537396..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServer.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.dictionary.server;
-
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-import org.apache.carbondata.core.dictionary.server.DictionaryServer;
-import org.apache.carbondata.core.dictionary.service.AbstractDictionaryServer;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-import com.google.common.collect.Lists;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.log4j.Logger;
-import org.apache.spark.SecurityManager;
-import org.apache.spark.SparkConf;
-import org.apache.spark.network.TransportContext;
-import org.apache.spark.network.netty.SparkTransportConf;
-import org.apache.spark.network.sasl.SaslServerBootstrap;
-import org.apache.spark.network.server.TransportServerBootstrap;
-import org.apache.spark.network.util.TransportConf;
-import scala.Some;
-
-/**
- * Dictionary Server to generate dictionary keys.
- */
-public class SecureDictionaryServer extends AbstractDictionaryServer implements DictionaryServer  {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(SecureDictionaryServer.class.getName());
-
-  private SecureDictionaryServerHandler secureDictionaryServerHandler;
-
-  private EventLoopGroup boss;
-  private EventLoopGroup worker;
-  private int port;
-  private String host;
-  private SparkConf conf;
-  private String secretKey = null;
-  private boolean encryptSecureServer;
-  private static Object lock = new Object();
-  private static SecureDictionaryServer INSTANCE = null;
-
-  private SecureDictionaryServer(SparkConf conf, String host, int port) {
-    this.conf = conf;
-    this.host = host;
-    this.port = port;
-    try {
-      UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          startServer();
-          return null;
-        }
-      });
-    } catch (IOException | InterruptedException io) {
-      LOGGER.error("Failed to start Dictionary Server in secure mode", io);
-    }
-  }
-
-  public static synchronized DictionaryServer getInstance(SparkConf conf, String host, int port,
-      CarbonTable carbonTable) throws Exception {
-    if (INSTANCE == null) {
-      INSTANCE = new SecureDictionaryServer(conf, host, port);
-    }
-    INSTANCE.initializeDictionaryGenerator(carbonTable);
-    return INSTANCE;
-  }
-
-  /**
-   * start dictionary server
-   *
-   */
-  @Override
-  public void startServer() {
-    LOGGER.info("Starting Dictionary Server in Secure Mode");
-    secureDictionaryServerHandler = new SecureDictionaryServerHandler();
-    String workerThreads = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.DICTIONARY_WORKER_THREADS,
-            CarbonCommonConstants.DICTIONARY_WORKER_THREADS_DEFAULT);
-    boss = new NioEventLoopGroup(1);
-    worker = new NioEventLoopGroup(Integer.parseInt(workerThreads));
-    // Configure the server.
-    bindToPort();
-  }
-
-  /**
-   * Binds dictionary server to an available port.
-   *
-   */
-  @Override
-  public void bindToPort() {
-    long start = System.currentTimeMillis();
-    // Configure the server.
-    int i = 0;
-    while (i < 10) {
-      int newPort = port + i;
-      try {
-        SecurityManager securityManager;
-        SparkConf conf = this.conf.clone();
-        conf.setAppName("Carbon Dictionary Server");
-
-        // As spark.network.sasl.serverAlwaysEncrypt is not exposed parameter
-        // set it explicitly so that Dictionary Server and Client communication
-        // occurs encrypted. The below parameter can be removed once spark Documents it.
-        // conf.set("spark.network.sasl.serverAlwaysEncrypt", "true");
-        conf.set("spark.authenticate.enableSaslEncryption", "true");
-
-        if (conf.get("spark.authenticate.enableSaslEncryption", "false").equalsIgnoreCase("true")) {
-          setEncryptSecureServer(true);
-        } else {
-          setEncryptSecureServer(false);
-        }
-
-        TransportConf transportConf =
-            SparkTransportConf.fromSparkConf(conf, "Carbon Dictionary Server", 0);
-        securityManager = new SecurityManager(conf, Some.<byte[]>empty());
-        secretKey = securityManager.getSecretKey();
-        TransportContext context =
-            new TransportContext(transportConf, secureDictionaryServerHandler);
-        TransportServerBootstrap bootstrap =
-            new SaslServerBootstrap(transportConf, securityManager);
-        String host = findLocalIpAddress(LOGGER);
-        //iteratively listening to newports
-        context
-            .createServer(host, newPort, Lists.<TransportServerBootstrap>newArrayList(bootstrap));
-        LOGGER.info(
-            "Dictionary Server started, Time spent " + (System.currentTimeMillis() - start)
-            + " Listening on port " + newPort);
-        this.port = newPort;
-        this.host = host;
-        break;
-      } catch (Exception e) {
-        LOGGER.error("Dictionary Server Failed to bind to port: " + newPort, e);
-        if (i == 9) {
-          throw new RuntimeException("Dictionary Server Could not bind to any port");
-        }
-      }
-      i++;
-    }
-  }
-
-  private void setEncryptSecureServer(boolean encryptSecureServer) {
-    this.encryptSecureServer = encryptSecureServer;
-  }
-
-  /**
-   * @return Port on which the SecureDictionaryServer has started.
-   */
-  @Override
-  public int getPort() {
-    return port;
-  }
-
-  /**
-   * @return IP address on which the SecureDictionaryServer has Started.
-   */
-  @Override
-  public String getHost() {
-    return host;
-  }
-
-  /**
-   * @return Secret Key of Dictionary Server.
-   */
-  @Override
-  public String getSecretKey() {
-    return secretKey;
-  }
-
-  @Override
-  public boolean isEncryptSecureServer() {
-    return encryptSecureServer;
-  }
-
-  /**
-   * shutdown dictionary server
-   *
-   * @throws Exception
-   */
-  @Override
-  public void shutdown() throws Exception {
-    LOGGER.info("Shutting down dictionary server");
-    try {
-      UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          worker.shutdownGracefully();
-          boss.shutdownGracefully();
-          return null;
-        }
-      });
-    } catch (IOException | InterruptedException e) {
-      LOGGER.error("Failed to stop Dictionary Server in secure mode", e);
-    }
-  }
-
-  public void initializeDictionaryGenerator(CarbonTable carbonTable) {
-    secureDictionaryServerHandler.initializeTable(carbonTable);
-  }
-
-  /**
-   * Write Dictionary for one table.
-   *
-   * @throws Exception
-   */
-
-  @Override
-  public void writeTableDictionary(String uniqueTableName) throws Exception {
-    DictionaryMessage key = new DictionaryMessage();
-    key.setTableUniqueId(uniqueTableName);
-    key.setType(DictionaryMessageType.WRITE_TABLE_DICTIONARY);
-    secureDictionaryServerHandler.processMessage(key);
-  }
-
-}
\ No newline at end of file
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServerHandler.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServerHandler.java
deleted file mode 100644
index 30807b6..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/dictionary/server/SecureDictionaryServerHandler.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.dictionary.server;
-
-import java.nio.ByteBuffer;
-
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.dictionary.generator.ServerDictionaryGenerator;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.Unpooled;
-import io.netty.channel.ChannelHandler;
-import org.apache.log4j.Logger;
-import org.apache.spark.network.client.RpcResponseCallback;
-import org.apache.spark.network.client.TransportClient;
-import org.apache.spark.network.server.OneForOneStreamManager;
-import org.apache.spark.network.server.RpcHandler;
-import org.apache.spark.network.server.StreamManager;
-
-/**
- * Handler for Dictionary server.
- */
-@ChannelHandler.Sharable public class SecureDictionaryServerHandler extends RpcHandler {
-
-  private static final Logger LOGGER =
-      LogServiceFactory.getLogService(SecureDictionaryServerHandler.class.getName());
-
-  /**
-   * dictionary generator
-   */
-  private ServerDictionaryGenerator generatorForServer = new ServerDictionaryGenerator();
-
-  /**
-   * process message by message type
-   *
-   * @param key
-   * @return
-   * @throws Exception
-   */
-  public int processMessage(DictionaryMessage key) throws Exception {
-    switch (key.getType()) {
-      case DICT_GENERATION:
-        generatorForServer.initializeGeneratorForColumn(key);
-        return generatorForServer.generateKey(key);
-      case SIZE:
-        generatorForServer.initializeGeneratorForColumn(key);
-        return generatorForServer.size(key);
-      case WRITE_TABLE_DICTIONARY:
-        generatorForServer.writeTableDictionaryData(key.getTableUniqueId());
-        return 0;
-      default:
-        return -1;
-    }
-  }
-
-  @Override
-  public void receive(TransportClient transportClient, ByteBuffer byteBuffer,
-      RpcResponseCallback rpcResponseCallback) {
-    try {
-      ByteBuf data = Unpooled.wrappedBuffer(byteBuffer);
-      DictionaryMessage key = new DictionaryMessage();
-      key.readFullLength(data);
-      data.release();
-      int outPut = processMessage(key);
-      key.setDictionaryValue(outPut);
-      // Send back the response
-      ByteBuf buff = ByteBufAllocator.DEFAULT.buffer();
-      key.writeData(buff);
-      rpcResponseCallback.onSuccess(buff.nioBuffer());
-    } catch (Exception e) {
-      LOGGER.error(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public StreamManager getStreamManager() {
-    return new OneForOneStreamManager();
-  }
-
-  public void initializeTable(CarbonTable carbonTable) {
-    generatorForServer.initializeGeneratorForTable(carbonTable);
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
index a22c8cb..5a41b50 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
@@ -46,16 +46,10 @@
     }
   }
 
-  lazy val singlePass: Boolean = options.getOrElse("single_pass", "false").toBoolean
-
   lazy val sortColumns: Option[String] = options.get("sort_columns")
 
   lazy val sortScope: Option[String] = options.get("sort_scope")
 
-  lazy val dictionaryInclude: Option[String] = options.get("dictionary_include")
-
-  lazy val dictionaryExclude: Option[String] = options.get("dictionary_exclude")
-
   lazy val longStringColumns: Option[String] = options.get("long_string_columns")
 
   lazy val tableBlockSize: Option[String] = options.get("table_blocksize")
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
deleted file mode 100644
index bb2dc91..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.rdd
-
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
-import org.apache.carbondata.core.statusmanager.SegmentStatus
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.spark.util.GlobalDictionaryUtil
-
-/**
- * This is a partitioner class for dividing the newly added columns into partitions
- *
- * @param rddId
- * @param idx
- * @param schema
- */
-class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Partition {
-  override def index: Int = idx
-
-  override def hashCode(): Int = 41 * (41 + rddId) + idx
-
-  val columnSchema = schema
-}
-
-/**
- * This class is aimed at generating dictionary file for the newly added columns
- */
-class AlterTableAddColumnRDD[K, V](
-    @transient private val sparkSession: SparkSession,
-    @transient private val newColumns: Seq[ColumnSchema],
-    identifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sparkSession, Nil) {
-
-  val lockType: String = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
-    CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)
-
-  override def internalGetPartitions: Array[Partition] = {
-    newColumns.zipWithIndex.map { column =>
-      new AddColumnPartition(id, column._2, column._1)
-    }.toArray
-  }
-
-  override def internalCompute(split: Partition,
-      context: TaskContext): Iterator[(Int, SegmentStatus)] = {
-    val status = SegmentStatus.SUCCESS
-    val iter = new Iterator[(Int, SegmentStatus)] {
-      try {
-        val columnSchema = split.asInstanceOf[AddColumnPartition].columnSchema
-        // create dictionary file if it is a dictionary column
-        if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
-            !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          var rawData: String = null
-          if (null != columnSchema.getDefaultValue) {
-            rawData = new String(columnSchema.getDefaultValue,
-              CarbonCommonConstants.DEFAULT_CHARSET_CLASS)
-          }
-          CarbonProperties.getInstance.addProperty(CarbonCommonConstants.LOCK_TYPE, lockType)
-          // Create table and metadata folders if not exist
-          val metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
-          if (!FileFactory.isFileExist(metadataDirectoryPath)) {
-            FileFactory.mkdirs(metadataDirectoryPath)
-          }
-          GlobalDictionaryUtil.loadDefaultDictionaryValueForNewColumn(
-            columnSchema,
-            identifier,
-            rawData)
-        }
-      } catch {
-        case ex: Exception =>
-          throw ex
-      }
-
-      var finished = false
-
-      override def hasNext: Boolean = {
-
-        if (!finished) {
-          finished = true
-          finished
-        } else {
-          !finished
-        }
-      }
-
-      override def next(): (Int, SegmentStatus) = {
-        (split.index, status)
-      }
-    }
-    iter
-  }
-
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
deleted file mode 100644
index b3eb4f5..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.rdd
-
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
-import org.apache.carbondata.core.statusmanager.SegmentStatus
-
-/**
- * This is a partitioner class for dividing the newly added columns into partitions
- *
- * @param rddId
- * @param idx
- * @param schema
- */
-class DropColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Partition {
-  override def index: Int = idx
-
-  override def hashCode(): Int = 41 * (41 + rddId) + idx
-
-  val columnSchema = schema
-}
-
-/**
- * This class is aimed at generating dictionary file for the newly added columns
- */
-class AlterTableDropColumnRDD[K, V](
-    @transient private val ss: SparkSession,
-    @transient private val newColumns: Seq[ColumnSchema],
-    carbonTableIdentifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](ss, Nil) {
-
-  override def internalGetPartitions: Array[Partition] = {
-    newColumns.zipWithIndex.map { column =>
-      new DropColumnPartition(id, column._2, column._1)
-    }.toArray
-  }
-
-  override def internalCompute(split: Partition,
-      context: TaskContext): Iterator[(Int, SegmentStatus)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val status = SegmentStatus.SUCCESS
-    val iter = new Iterator[(Int, SegmentStatus)] {
-      try {
-        val columnSchema = split.asInstanceOf[DropColumnPartition].columnSchema
-        if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
-            !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          ManageDictionaryAndBTree
-            .deleteDictionaryFileAndCache(columnSchema, carbonTableIdentifier)
-        }
-      } catch {
-        case ex: Exception =>
-          LOGGER.error(ex.getMessage, ex)
-          throw ex
-      }
-
-      var finished = false
-
-      override def hasNext: Boolean = {
-
-        if (!finished) {
-          finished = true
-          finished
-        } else {
-          !finished
-        }
-      }
-
-      override def next(): (Int, SegmentStatus) = {
-        (split.index, status)
-      }
-    }
-    iter
-  }
-
-}
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 8168386..86f28ca 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
@@ -17,37 +17,19 @@
 
 package org.apache.carbondata.spark.rdd
 
-import java.io.{DataInputStream, InputStreamReader}
-import java.nio.charset.Charset
-import java.text.SimpleDateFormat
-import java.util
 import java.util.regex.Pattern
 
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
-import scala.util.control.Breaks.breakable
 
-import au.com.bytecode.opencsv.CSVReader
-import com.univocity.parsers.common.TextParsingException
 import org.apache.commons.lang3.{ArrayUtils, StringUtils}
 import org.apache.spark._
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.Row
 
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.constants.CarbonLoadOptionConstants
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
-import org.apache.carbondata.core.statusmanager.SegmentStatus
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.processing.util.CarbonLoaderUtil
-import org.apache.carbondata.spark.tasks.{DictionaryWriterTask, SortIndexWriterTask}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
 
 /**
  * A partitioner partition by column.
@@ -147,7 +129,8 @@
 /**
  * a case class to package some attributes
  */
-case class DictionaryLoadModel(table: AbsoluteTableIdentifier,
+case class DictionaryLoadModel(
+    table: AbsoluteTableIdentifier,
     dimensions: Array[CarbonDimension],
     hdfsLocation: String,
     dictfolderPath: String,
@@ -167,70 +150,6 @@
 
 case class ColumnDistinctValues(values: Array[String], rowCount: Long) extends Serializable
 
-/**
- * A RDD to combine all dictionary distinct values.
- *
- * @constructor create a RDD with RDD[(String, Iterable[String])]
- * @param prev  the input RDD[(String, Iterable[String])]
- * @param model a model package load info
- */
-class CarbonAllDictionaryCombineRDD(
-    @transient private val sparkSession: SparkSession,
-    prev: RDD[(String, Iterable[String])],
-    model: DictionaryLoadModel)
-  extends CarbonRDD[(Int, ColumnDistinctValues)](sparkSession, prev) {
-
-  override def internalGetPartitions: Array[Partition] = {
-    firstParent[(String, Iterable[String])].partitions
-  }
-
-    override def internalCompute(split: Partition, context: TaskContext
-  ): Iterator[(Int, ColumnDistinctValues)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val distinctValuesList = new ArrayBuffer[(Int, mutable.HashSet[String])]
-    /*
-     * for all dictionary, all columns need to encoding and checking
-     * isHighCardinalityColumn, so no need to calculate rowcount
-     */
-    val rowCount = 0L
-    try {
-      val dimensionParsers =
-        GlobalDictionaryUtil.createDimensionParsers(model, distinctValuesList)
-      val dimNum = model.dimensions.length
-      // Map[dimColName -> dimColNameIndex]
-      val columnIndexMap = new mutable.HashMap[String, Int]()
-      for (j <- 0 until dimNum) {
-        columnIndexMap.put(model.dimensions(j).getColName, j)
-      }
-
-      var row: (String, Iterable[String]) = null
-      val rddIter = firstParent[(String, Iterable[String])].iterator(split, context)
-      // generate block distinct value set
-      while (rddIter.hasNext) {
-        row = rddIter.next()
-        if (row != null) {
-          columnIndexMap.get(row._1) match {
-            case Some(index) =>
-              for (record <- row._2) {
-                dimensionParsers(index).parseString(record)
-              }
-            case None =>
-          }
-        }
-      }
-    } catch {
-      case ex: Exception =>
-        LOGGER.error(ex)
-        throw ex
-    }
-
-    distinctValuesList.map { iter =>
-      val valueList = iter._2.toArray
-      (iter._1, ColumnDistinctValues(valueList, rowCount))
-    }.iterator
-  }
-}
-
 class StringArrayRow(var values: Array[String]) extends Row {
 
   override def length: Int = values.length
@@ -260,322 +179,3 @@
     this
   }
 }
-
-/**
- * A RDD to combine distinct values in block.
- *
- * @constructor create a RDD with RDD[Row]
- * @param prev  the input RDD[Row]
- * @param model a model package load info
- */
-class CarbonBlockDistinctValuesCombineRDD(
-    @transient private val ss: SparkSession,
-    prev: RDD[Row],
-    model: DictionaryLoadModel)
-  extends CarbonRDD[(Int, ColumnDistinctValues)](ss, prev) {
-
-  override def internalGetPartitions: Array[Partition] = firstParent[Row].partitions
-  override def internalCompute(split: Partition,
-      context: TaskContext): Iterator[(Int, ColumnDistinctValues)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordLoadCsvfilesToDfTime()
-    val distinctValuesList = new ArrayBuffer[(Int, mutable.HashSet[String])]
-    var rowCount = 0L
-    try {
-      val dimensionParsers =
-        GlobalDictionaryUtil.createDimensionParsers(model, distinctValuesList)
-      val dimNum = model.dimensions.length
-      var row: Row = null
-      val rddIter = firstParent[Row].iterator(split, context)
-      val timeStampFormat = new SimpleDateFormat(model.defaultTimestampFormat)
-      val dateFormat = new SimpleDateFormat(model.defaultDateFormat)
-      // generate block distinct value set
-      while (rddIter.hasNext) {
-        row = rddIter.next()
-        if (row != null) {
-          rowCount += 1
-          val complexDelimiters = new util.ArrayList[String]
-          model.delimiters.foreach(x => complexDelimiters.add(x))
-          for (i <- 0 until dimNum) {
-            dimensionParsers(i).parseString(CarbonScalaUtil.getString(row.get(i),
-              model.serializationNullFormat,
-              complexDelimiters,
-              timeStampFormat,
-              dateFormat))
-          }
-        }
-      }
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordLoadCsvfilesToDfTime()
-    } catch {
-      case txe: TextParsingException =>
-        throw txe
-      case ex: Exception =>
-        LOGGER.error(ex)
-        throw ex
-    }
-
-    distinctValuesList.map { iter =>
-      val valueList = iter._2.toArray
-      (iter._1, ColumnDistinctValues(valueList, rowCount))
-    }.iterator
-  }
-}
-
-/**
- * A RDD to generate dictionary file for each column
- *
- * @constructor create a RDD with RDD[Row]
- * @param prev  the input RDD[Row]
- * @param model a model package load info
- */
-class CarbonGlobalDictionaryGenerateRDD(
-    @transient private val sparkSession: SparkSession,
-    prev: RDD[(Int, ColumnDistinctValues)],
-    model: DictionaryLoadModel)
-  extends CarbonRDD[(Int, SegmentStatus)](sparkSession, prev) {
-
-  override def internalGetPartitions: Array[Partition] =
-    firstParent[(Int, ColumnDistinctValues)].partitions
-
-
-  override def internalCompute(split: Partition,
-      context: TaskContext): Iterator[(Int, SegmentStatus)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    var status = SegmentStatus.SUCCESS
-    val iter = new Iterator[(Int, SegmentStatus)] {
-      var dictionaryForDistinctValueLookUp: Dictionary = _
-      var dictionaryForDistinctValueLookUpCleared: Boolean = false
-      val dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier = new
-          DictionaryColumnUniqueIdentifier(
-        model.table,
-        model.columnIdentifier(split.index),
-        model.columnIdentifier(split.index).getDataType)
-      if (StringUtils.isNotBlank(model.hdfsTempLocation)) {
-        CarbonProperties.getInstance.addProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION,
-          model.hdfsTempLocation)
-      }
-      if (StringUtils.isNotBlank(model.lockType)) {
-        CarbonProperties.getInstance.addProperty(CarbonCommonConstants.LOCK_TYPE,
-          model.lockType)
-      }
-      if (StringUtils.isNotBlank(model.zooKeeperUrl)) {
-        CarbonProperties.getInstance.addProperty(CarbonCommonConstants.ZOOKEEPER_URL,
-          model.zooKeeperUrl)
-      }
-      val dictLock: ICarbonLock = CarbonLockFactory
-        .getCarbonLockObj(model.table,
-          model.columnIdentifier(split.index).getColumnId + LockUsage.LOCK)
-      var isDictionaryLocked = false
-      // generate distinct value list
-      try {
-        val t1 = System.currentTimeMillis
-        val valuesBuffer = new mutable.HashSet[String]
-        val rddIter = firstParent[(Int, ColumnDistinctValues)].iterator(split, context)
-        var rowCount = 0L
-        CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordDicShuffleAndWriteTime()
-        breakable {
-          while (rddIter.hasNext) {
-            val distinctValueList = rddIter.next()._2
-            valuesBuffer ++= distinctValueList.values
-            rowCount += distinctValueList.rowCount
-          }
-        }
-        val combineListTime = System.currentTimeMillis() - t1
-        isDictionaryLocked = dictLock.lockWithRetries()
-        if (isDictionaryLocked) {
-          logInfo(s"Successfully able to get the dictionary lock for ${
-            model.primDimensions(split.index).getColName
-          }")
-        } else {
-          sys.error(s"Dictionary file ${
-              model.primDimensions(split.index).getColName
-          } is locked for updation. Please try after some time")
-        }
-        val t2 = System.currentTimeMillis
-        val isDictFileExists = FileFactory.isFileExist(model.dictFilePaths(split.index))
-        dictionaryForDistinctValueLookUp = if (isDictFileExists) {
-          CarbonLoaderUtil.getDictionary(model.table,
-            model.columnIdentifier(split.index),
-            model.primDimensions(split.index).getDataType
-          )
-        } else {
-          null
-        }
-        val dictCacheTime = System.currentTimeMillis - t2
-        val t3 = System.currentTimeMillis()
-        val dictWriteTask = new DictionaryWriterTask(valuesBuffer,
-          dictionaryForDistinctValueLookUp,
-          dictionaryColumnUniqueIdentifier,
-          model.primDimensions(split.index).getColumnSchema,
-          isDictFileExists
-        )
-        // execute dictionary writer task to get distinct values
-        val distinctValues = dictWriteTask.execute()
-        val dictWriteTime = System.currentTimeMillis() - t3
-        val t4 = System.currentTimeMillis()
-        // if new data came than rewrite sort index file
-        if (distinctValues.size() > 0) {
-          val sortIndexWriteTask = new SortIndexWriterTask(dictionaryColumnUniqueIdentifier,
-            model.primDimensions(split.index).getDataType,
-            dictionaryForDistinctValueLookUp,
-            distinctValues)
-          sortIndexWriteTask.execute()
-        }
-        val sortIndexWriteTime = System.currentTimeMillis() - t4
-        CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordDicShuffleAndWriteTime()
-        // After sortIndex writing, update dictionaryMeta
-        dictWriteTask.updateMetaData()
-        // clear the value buffer after writing dictionary data
-        valuesBuffer.clear
-        CarbonUtil.clearDictionaryCache(dictionaryForDistinctValueLookUp)
-        dictionaryForDistinctValueLookUpCleared = true
-        LOGGER.info(s"\n columnName: ${ model.primDimensions(split.index).getColName }" +
-                    s"\n columnId: ${ model.primDimensions(split.index).getColumnId }" +
-                    s"\n new distinct values count: ${ distinctValues.size() }" +
-                    s"\n combine lists: $combineListTime" +
-                    s"\n create dictionary cache: $dictCacheTime" +
-                    s"\n sort list, distinct and write: $dictWriteTime" +
-                    s"\n write sort info: $sortIndexWriteTime")
-      } catch {
-        case dictionaryException: NoRetryException =>
-          LOGGER.error(dictionaryException)
-          status = SegmentStatus.LOAD_FAILURE
-        case ex: Exception =>
-          LOGGER.error(ex)
-          throw ex
-      } finally {
-        if (!dictionaryForDistinctValueLookUpCleared) {
-          CarbonUtil.clearDictionaryCache(dictionaryForDistinctValueLookUp)
-        }
-        if (dictLock != null && isDictionaryLocked) {
-          if (dictLock.unlock()) {
-            logInfo(s"Dictionary ${
-              model.primDimensions(split.index).getColName
-            } Unlocked Successfully.")
-          } else {
-            logError(s"Unable to unlock Dictionary ${
-              model.primDimensions(split.index).getColName
-            }")
-          }
-        }
-      }
-      var finished = false
-
-      override def hasNext: Boolean = {
-
-        if (!finished) {
-          finished = true
-          finished
-        } else {
-          !finished
-        }
-      }
-
-      override def next(): (Int, SegmentStatus) = {
-        (split.index, status)
-      }
-    }
-
-    iter
-  }
-
-}
-
-/**
- * Set column dictionary partition format
- *
- * @param id        partition id
- * @param dimension current carbon dimension
- */
-class CarbonColumnDictPartition(id: Int, dimension: CarbonDimension)
-  extends Partition {
-  override val index: Int = id
-  val preDefDictDimension: CarbonDimension = dimension
-}
-
-
-/**
- * Use external column dict to generate global dictionary
- *
- * @param carbonLoadModel carbon load model
- * @param table           carbon table identifier
- * @param dimensions      carbon dimensions having predefined dict
- * @param dictFolderPath  path of dictionary folder
- */
-class CarbonColumnDictGenerateRDD(
-    carbonLoadModel: CarbonLoadModel,
-    dictionaryLoadModel: DictionaryLoadModel,
-    @transient private val ss: SparkSession,
-    table: CarbonTableIdentifier,
-    dimensions: Array[CarbonDimension],
-    dictFolderPath: String)
-  extends CarbonRDD[(Int, ColumnDistinctValues)](ss, Nil) {
-
-  override def internalGetPartitions: Array[Partition] = {
-    val primDimensions = dictionaryLoadModel.primDimensions
-    val primDimLength = primDimensions.length
-    val result = new Array[Partition](primDimLength)
-    for (i <- 0 until primDimLength) {
-      result(i) = new CarbonColumnDictPartition(i, primDimensions(i))
-    }
-    result
-  }
-
-  override def internalCompute(split: Partition, context: TaskContext)
-  : Iterator[(Int, ColumnDistinctValues)] = {
-    val theSplit = split.asInstanceOf[CarbonColumnDictPartition]
-    val primDimension = theSplit.preDefDictDimension
-    // read the column dict data
-    val preDefDictFilePath = carbonLoadModel.getPredefDictFilePath(primDimension)
-    var csvReader: CSVReader = null
-    var inputStream: DataInputStream = null
-    var colDictData: java.util.Iterator[Array[String]] = null
-    try {
-      inputStream = FileFactory.getDataInputStream(preDefDictFilePath)
-      csvReader = new CSVReader(new InputStreamReader(inputStream, Charset.defaultCharset),
-        carbonLoadModel.getCsvDelimiter.charAt(0))
-      // read the column data to list iterator
-      colDictData = csvReader.readAll.iterator
-    } catch {
-      case ex: Exception =>
-        logError(s"Error in reading pre-defined " +
-                 s"dictionary file:${ ex.getMessage }")
-        throw ex
-    } finally {
-      if (csvReader != null) {
-        try {
-          csvReader.close()
-        } catch {
-          case ex: Exception =>
-            logError(s"Error in closing csvReader of " +
-                     s"pre-defined dictionary file:${ ex.getMessage }")
-        }
-      }
-      if (inputStream != null) {
-        try {
-          inputStream.close()
-        } catch {
-          case ex: Exception =>
-            logError(s"Error in closing inputStream of " +
-                     s"pre-defined dictionary file:${ ex.getMessage }")
-        }
-      }
-    }
-    val mapIdWithSet = new mutable.HashMap[String, mutable.HashSet[String]]
-    val columnValues = new mutable.HashSet[String]
-    val distinctValues = (theSplit.index, columnValues)
-    mapIdWithSet.put(primDimension.getColumnId, columnValues)
-    // use parser to generate new dict value
-    val dimensionParser = GlobalDictionaryUtil.generateParserForDimension(
-      Some(primDimension),
-      GlobalDictionaryUtil.createDataFormat(carbonLoadModel.getDelimiters),
-      mapIdWithSet).get
-    // parse the column data
-    while (colDictData.hasNext) {
-      dimensionParser.parseString(colDictData.next()(0))
-    }
-    Array((distinctValues._1,
-      ColumnDistinctValues(distinctValues._2.toArray, 0L))).iterator
-  }
-
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala
deleted file mode 100644
index 08d635b..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/DictionaryWriterTask.scala
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.tasks
-
-import java.io.IOException
-
-import scala.collection.mutable
-
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
-import org.apache.carbondata.core.service.CarbonCommonFactory
-import org.apache.carbondata.core.util.DataTypeUtil
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter
-
-/**
- *
- * @param valuesBuffer
- * @param dictionary
- * @param dictionaryColumnUniqueIdentifier
- * @param columnSchema
- * @param isDictionaryFileExist
- * @param writer
- */
-class DictionaryWriterTask(valuesBuffer: mutable.HashSet[String],
-    dictionary: Dictionary,
-    dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier,
-    columnSchema: ColumnSchema,
-    isDictionaryFileExist: Boolean,
-    var writer: CarbonDictionaryWriter = null) {
-
-  /**
-   * execute the task
-   *
-   * @return distinctValueList and time taken to write
-   */
-  def execute(): java.util.List[String] = {
-    val values = valuesBuffer.toArray
-    java.util.Arrays.sort(values, Ordering[String])
-    val dictService = CarbonCommonFactory.getDictionaryService
-    writer = dictService.getDictionaryWriter(dictionaryColumnUniqueIdentifier)
-    val distinctValues: java.util.List[String] = new java.util.ArrayList()
-
-    try {
-      if (!isDictionaryFileExist) {
-        writer.write(CarbonCommonConstants.MEMBER_DEFAULT_VAL)
-        distinctValues.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL)
-      }
-
-      if (values.length >= 1) {
-        if (isDictionaryFileExist) {
-          for (value <- values) {
-            val parsedValue = DataTypeUtil.normalizeColumnValueForItsDataType(value,
-              columnSchema)
-            if (null != parsedValue && dictionary.getSurrogateKey(parsedValue) ==
-              CarbonCommonConstants.INVALID_SURROGATE_KEY) {
-              writer.write(parsedValue)
-              distinctValues.add(parsedValue)
-            }
-          }
-
-        } else {
-          for (value <- values) {
-            val parsedValue = DataTypeUtil.normalizeColumnValueForItsDataType(value,
-              columnSchema)
-            if (null != parsedValue) {
-              writer.write(parsedValue)
-              distinctValues.add(parsedValue)
-            }
-          }
-        }
-      }
-    } catch {
-      case ex: IOException =>
-        throw ex
-    } finally {
-      if (null != writer) {
-        writer.close()
-      }
-    }
-    distinctValues
-  }
-
-  /**
-   * update dictionary metadata
-   */
-  def updateMetaData() {
-    if (null != writer) {
-      writer.commit()
-    }
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala
deleted file mode 100644
index 2d8ed1d..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/tasks/SortIndexWriterTask.scala
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.tasks
-
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.metadata.datatype.DataType
-import org.apache.carbondata.core.service.CarbonCommonFactory
-import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
-
-/**
- * This task writes sort index file
- *
- * @param dictionaryColumnUniqueIdentifier
- * @param dataType
- * @param dictionary
- * @param distinctValues
- * @param carbonDictionarySortIndexWriter
- */
-class SortIndexWriterTask(
-    dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier,
-    dataType: DataType,
-    dictionary: Dictionary,
-    distinctValues: java.util.List[String],
-    var carbonDictionarySortIndexWriter: CarbonDictionarySortIndexWriter = null) {
-  def execute() {
-    try {
-      if (distinctValues.size() > 0) {
-        val preparator: CarbonDictionarySortInfoPreparator = new CarbonDictionarySortInfoPreparator
-        val dictService = CarbonCommonFactory.getDictionaryService
-        val dictionarySortInfo: CarbonDictionarySortInfo =
-          preparator.getDictionarySortInfo(distinctValues, dictionary,
-            dataType)
-        carbonDictionarySortIndexWriter =
-          dictService
-            .getDictionarySortIndexWriter(dictionaryColumnUniqueIdentifier)
-        carbonDictionarySortIndexWriter.writeSortIndex(dictionarySortInfo.getSortIndex)
-        carbonDictionarySortIndexWriter
-          .writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted)
-      }
-    } finally {
-      if (null != carbonDictionarySortIndexWriter) {
-        carbonDictionarySortIndexWriter.close()
-      }
-    }
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index fd0cf74..3f040c9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -160,13 +160,9 @@
           return DateTimeUtils.dateToString(date.toString.toInt)
         }
       }
-      val dictionaryPath =
-        table.getTableInfo.getFactTable.getTableProperties.get(
-          CarbonCommonConstants.DICTIONARY_PATH)
       val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
         table.getAbsoluteTableIdentifier,
-        column.getColumnIdentifier, column.getDataType,
-        dictionaryPath)
+        column.getColumnIdentifier, column.getDataType)
       return forwardDictionaryCache.get(
         dictionaryColumnUniqueIdentifier).getDictionaryValueForKey(value.toInt)
     }
@@ -193,8 +189,7 @@
    */
   def convertStaticPartitions(
       value: String,
-      column: ColumnSchema,
-      table: CarbonTable): String = {
+      column: ColumnSchema): String = {
     try {
       if (column.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         if (column.getDataType.equals(CarbonDataTypes.TIMESTAMP)) {
@@ -208,22 +203,6 @@
             CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT
           ).generateDirectSurrogateKey(value).toString
         }
-      } else if (column.hasEncoding(Encoding.DICTIONARY)) {
-        val cacheProvider: CacheProvider = CacheProvider.getInstance
-        val reverseCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
-          cacheProvider.createCache(CacheType.REVERSE_DICTIONARY)
-        val dictionaryPath =
-          table.getTableInfo.getFactTable.getTableProperties.get(
-            CarbonCommonConstants.DICTIONARY_PATH)
-        val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
-          table.getAbsoluteTableIdentifier,
-          new ColumnIdentifier(
-            column.getColumnUniqueId,
-            column.getColumnProperties,
-            column.getDataType),
-          column.getDataType,
-          dictionaryPath)
-        return reverseCache.get(dictionaryColumnUniqueIdentifier).getSurrogateKey(value).toString
       }
       column.getDataType match {
         case CarbonDataTypes.TIMESTAMP =>
@@ -267,7 +246,7 @@
           (col, value)
         } else {
           val convertedString =
-            CarbonScalaUtil.convertToCarbonFormat(
+            convertToCarbonFormat(
               value,
               carbonColumn,
               forwardDictionaryCache,
@@ -498,8 +477,6 @@
    */
   def validateLocalDictionaryColumns(tableProperties: mutable.Map[String, String],
       localDictColumns: Seq[String]): Unit = {
-    var dictIncludeColumns: Seq[String] = Seq[String]()
-
     // check if the duplicate columns are specified in table schema
     if (localDictColumns.distinct.lengthCompare(localDictColumns.size) != 0) {
       val duplicateColumns = localDictColumns
@@ -510,24 +487,6 @@
         ". Please check the DDL."
       throw new MalformedCarbonCommandException(errMsg)
     }
-
-    // check if the same column is present in both dictionary include and local dictionary columns
-    // configuration
-    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
-      dictIncludeColumns =
-        tableProperties(CarbonCommonConstants.DICTIONARY_INCLUDE).split(",").map(_.trim)
-      localDictColumns.foreach { distCol =>
-        if (dictIncludeColumns.exists(x => x.equalsIgnoreCase(distCol.trim))) {
-          val commonColumn = (dictIncludeColumns ++ localDictColumns)
-            .diff((dictIncludeColumns ++ localDictColumns).distinct).distinct
-          val errorMsg = "LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: " +
-                         commonColumn.mkString(",") +
-                         " specified in Dictionary include. Local Dictionary will not be " +
-                         "generated for Dictionary include columns. Please check the DDL."
-          throw new MalformedCarbonCommandException(errorMsg)
-        }
-      }
-    }
   }
 
   /**
@@ -628,10 +587,8 @@
    */
   def validateLocalConfiguredDictionaryColumns(fields: Seq[Field],
       tableProperties: mutable.Map[String, String], localDictColumns: Seq[String]): Unit = {
-    var dictIncludeColumns: Seq[String] = Seq[String]()
-
     // validate the local dict columns
-    CarbonScalaUtil.validateLocalDictionaryColumns(tableProperties, localDictColumns)
+    validateLocalDictionaryColumns(tableProperties, localDictColumns)
     // check if the column specified exists in table schema
     localDictColumns.foreach { distCol =>
       if (!fields.exists(x => x.column.equalsIgnoreCase(distCol.trim))) {
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
deleted file mode 100644
index 6a338a5..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ /dev/null
@@ -1,857 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util
-
-import java.util.regex.Pattern
-
-import scala.collection.mutable
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import scala.language.implicitConversions
-import scala.util.control.Breaks.{break, breakable}
-
-import org.apache.commons.lang3.{ArrayUtils, StringUtils}
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
-import org.apache.hadoop.mapreduce.security.TokenCache
-import org.apache.spark.{Accumulator, SparkException}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.{NewHadoopRDD, RDD}
-import org.apache.spark.sql._
-import org.apache.spark.util.FileUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.column.{CarbonDimension, ColumnSchema}
-import org.apache.carbondata.core.statusmanager.SegmentStatus
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.processing.exception.DataLoadingException
-import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
-import org.apache.carbondata.processing.loading.exception.NoRetryException
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.spark.CarbonSparkFactory
-import org.apache.carbondata.spark.rdd._
-import org.apache.carbondata.spark.tasks.{DictionaryWriterTask, SortIndexWriterTask}
-
-/**
- * A object which provide a method to generate global dictionary from CSV files.
- */
-object GlobalDictionaryUtil {
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  /**
-   * The default separator to use if none is supplied to the constructor.
-   */
-  val DEFAULT_SEPARATOR: Char = ','
-  /**
-   * The default quote character to use if none is supplied to the
-   * constructor.
-   */
-  val DEFAULT_QUOTE_CHARACTER: Char = '"'
-
-  /**
-   * find columns which need to generate global dictionary.
-   *
-   * @param dimensions dimension list of schema
-   * @param headers    column headers
-   * @param columns    column list of csv file
-   */
-  def pruneDimensions(dimensions: Array[CarbonDimension],
-      headers: Array[String],
-      columns: Array[String]): (Array[CarbonDimension], Array[String]) = {
-    val dimensionBuffer = new ArrayBuffer[CarbonDimension]
-    val columnNameBuffer = new ArrayBuffer[String]
-    val dimensionsWithDict = dimensions.filter(hasEncoding(_, Encoding.DICTIONARY,
-      Encoding.DIRECT_DICTIONARY))
-    dimensionsWithDict.foreach { dim =>
-      breakable {
-        headers.zipWithIndex.foreach { h =>
-          if (dim.getColName.equalsIgnoreCase(h._1)) {
-            dimensionBuffer += dim
-            columnNameBuffer += columns(h._2)
-            break
-          }
-        }
-      }
-    }
-    (dimensionBuffer.toArray, columnNameBuffer.toArray)
-  }
-
-  /**
-   * use this method to judge whether CarbonDimension use some encoding or not
-   *
-   * @param dimension       carbonDimension
-   * @param encoding        the coding way of dimension
-   * @param excludeEncoding the coding way to exclude
-   */
-  def hasEncoding(dimension: CarbonDimension,
-      encoding: Encoding,
-      excludeEncoding: Encoding): Boolean = {
-    if (dimension.isComplex()) {
-      val children = dimension.getListOfChildDimensions
-      children.asScala.exists(hasEncoding(_, encoding, excludeEncoding))
-    } else {
-      dimension.hasEncoding(encoding) &&
-      (excludeEncoding == null || !dimension.hasEncoding(excludeEncoding))
-    }
-  }
-
-  def gatherDimensionByEncoding(carbonLoadModel: CarbonLoadModel,
-      dimension: CarbonDimension,
-      encoding: Encoding,
-      excludeEncoding: Encoding,
-      dimensionsWithEncoding: ArrayBuffer[CarbonDimension],
-      forPreDefDict: Boolean) {
-    if (dimension.isComplex) {
-      val children = dimension.getListOfChildDimensions.asScala
-      children.foreach { c =>
-        gatherDimensionByEncoding(carbonLoadModel, c, encoding, excludeEncoding,
-          dimensionsWithEncoding, forPreDefDict)
-      }
-    } else {
-      if (dimension.hasEncoding(encoding) &&
-          (excludeEncoding == null || !dimension.hasEncoding(excludeEncoding))) {
-        if ((forPreDefDict && carbonLoadModel.getPredefDictFilePath(dimension) != null) ||
-            (!forPreDefDict && carbonLoadModel.getPredefDictFilePath(dimension) == null)) {
-          dimensionsWithEncoding += dimension
-        }
-      }
-    }
-  }
-
-  def getPrimDimensionWithDict(carbonLoadModel: CarbonLoadModel,
-      dimension: CarbonDimension,
-      forPreDefDict: Boolean): Array[CarbonDimension] = {
-    val dimensionsWithDict = new ArrayBuffer[CarbonDimension]
-    gatherDimensionByEncoding(carbonLoadModel, dimension, Encoding.DICTIONARY,
-      Encoding.DIRECT_DICTIONARY,
-      dimensionsWithDict, forPreDefDict)
-    dimensionsWithDict.toArray
-  }
-
-  def generateParserForChildrenDimension(dim: CarbonDimension,
-      format: DataFormat,
-      mapColumnValuesWithId:
-      HashMap[String, HashSet[String]],
-      generic: GenericParser): Unit = {
-    val children = dim.getListOfChildDimensions.asScala
-    for (i <- children.indices) {
-      generateParserForDimension(Some(children(i)), format.cloneAndIncreaseIndex,
-        mapColumnValuesWithId) match {
-        case Some(childDim) =>
-          generic.addChild(childDim)
-        case None =>
-      }
-    }
-  }
-
-  def generateParserForDimension(dimension: Option[CarbonDimension],
-      format: DataFormat,
-      mapColumnValuesWithId: HashMap[String, HashSet[String]]): Option[GenericParser] = {
-    dimension match {
-      case None =>
-        None
-      case Some(dim) =>
-        if (DataTypes.isArrayType(dim.getDataType) || DataTypes.isMapType(dim.getDataType)) {
-          val arrDim = ArrayParser(dim, format)
-          generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, arrDim)
-          Some(arrDim)
-        } else if (DataTypes.isStructType(dim.getDataType)) {
-          val stuDim = StructParser(dim, format)
-          generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, stuDim)
-          Some(stuDim)
-        } else {
-          Some(PrimitiveParser(dim, mapColumnValuesWithId.get(dim.getColumnId)))
-        }
-    }
-  }
-
-  def createDataFormat(delimiters: Array[String]): DataFormat = {
-    if (ArrayUtils.isNotEmpty(delimiters)) {
-      val patterns = delimiters.map { d =>
-        Pattern.compile(if (d == null) {
-          ""
-        } else {
-          CarbonUtil.delimiterConverter(d)
-        })
-      }
-      DataFormat(delimiters.map(CarbonUtil.delimiterConverter(_)), 0, patterns)
-    } else {
-      null
-    }
-  }
-
-  /**
-   * create a instance of DictionaryLoadModel
-   *
-   * @param carbonLoadModel carbon load model
-   * @param table           CarbonTableIdentifier
-   * @param dimensions      column list
-   * @param dictFolderPath  path of dictionary folder
-   */
-  def createDictionaryLoadModel(
-      carbonLoadModel: CarbonLoadModel,
-      table: CarbonTableIdentifier,
-      dimensions: Array[CarbonDimension],
-      dictFolderPath: String,
-      forPreDefDict: Boolean): DictionaryLoadModel = {
-    val primDimensionsBuffer = new ArrayBuffer[CarbonDimension]
-    val isComplexes = new ArrayBuffer[Boolean]
-    for (i <- dimensions.indices) {
-      val dims = getPrimDimensionWithDict(carbonLoadModel, dimensions(i), forPreDefDict)
-      for (j <- dims.indices) {
-        primDimensionsBuffer += dims(j)
-        isComplexes += dimensions(i).isComplex
-      }
-    }
-    val primDimensions = primDimensionsBuffer.map { x => x }.toArray
-    val dictDetail = CarbonSparkFactory.getDictionaryDetailService.
-      getDictionaryDetail(dictFolderPath, primDimensions, carbonLoadModel.getTablePath)
-    val dictFilePaths = dictDetail.dictFilePaths
-    val dictFileExists = dictDetail.dictFileExists
-    val columnIdentifier = dictDetail.columnIdentifiers
-    val hdfsTempLocation = CarbonProperties.getInstance.
-      getProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION, System.getProperty("java.io.tmpdir"))
-    val lockType = CarbonProperties.getInstance
-      .getProperty(CarbonCommonConstants.LOCK_TYPE, CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)
-    val zookeeperUrl = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.ZOOKEEPER_URL)
-    val serializationNullFormat =
-      carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
-    // get load count
-    if (null == carbonLoadModel.getLoadMetadataDetails) {
-      carbonLoadModel.readAndSetLoadMetadataDetails()
-    }
-    val absoluteTableIdentifier = AbsoluteTableIdentifier.from(carbonLoadModel.getTablePath, table)
-    DictionaryLoadModel(
-      absoluteTableIdentifier,
-      dimensions,
-      carbonLoadModel.getTablePath,
-      dictFolderPath,
-      dictFilePaths,
-      dictFileExists,
-      isComplexes.toArray,
-      primDimensions,
-      carbonLoadModel.getDelimiters,
-      columnIdentifier,
-      carbonLoadModel.getLoadMetadataDetails.size() == 0,
-      hdfsTempLocation,
-      lockType,
-      zookeeperUrl,
-      serializationNullFormat,
-      carbonLoadModel.getDefaultTimestampFormat,
-      carbonLoadModel.getDefaultDateFormat)
-  }
-
-  /**
-   * load and prune dictionary Rdd from csv file or input dataframe
-   *
-   * @param sqlContext sqlContext
-   * @param carbonLoadModel carbonLoadModel
-   * @param inputDF input dataframe
-   * @param requiredCols names of dictionary column
-   * @param hadoopConf hadoop configuration
-   * @return rdd that contains only dictionary columns
-   */
-  private def loadInputDataAsDictRdd(sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      inputDF: Option[DataFrame],
-      requiredCols: Array[String],
-      hadoopConf: Configuration): RDD[Row] = {
-    if (inputDF.isDefined) {
-      inputDF.get.select(requiredCols.head, requiredCols.tail : _*).rdd
-    } else {
-      CommonUtil.configureCSVInputFormat(hadoopConf, carbonLoadModel)
-      hadoopConf.set(FileInputFormat.INPUT_DIR, carbonLoadModel.getFactFilePath)
-      val headerCols = carbonLoadModel.getCsvHeaderColumns.map(_.toLowerCase)
-      val header2Idx = headerCols.zipWithIndex.toMap
-      // index of dictionary columns in header
-      val dictColIdx = requiredCols.map(c => header2Idx(c.toLowerCase))
-
-      val jobConf = new JobConf(hadoopConf)
-      SparkHadoopUtil.get.addCredentials(jobConf)
-      TokenCache.obtainTokensForNamenodes(jobConf.getCredentials,
-        Array[Path](new Path(carbonLoadModel.getFactFilePath)),
-        jobConf)
-      val dictRdd = new NewHadoopRDD[NullWritable, StringArrayWritable](
-        sqlContext.sparkContext,
-        classOf[CSVInputFormat],
-        classOf[NullWritable],
-        classOf[StringArrayWritable],
-        jobConf)
-        .setName("global dictionary")
-        .map[Row] { currentRow =>
-        val rawRow = currentRow._2.get()
-        val destRow = new Array[String](dictColIdx.length)
-        for (i <- dictColIdx.indices) {
-          // dictionary index in this row
-          val idx = dictColIdx(i)
-          // copy specific dictionary value from source to dest
-          if (idx < rawRow.length) {
-            System.arraycopy(rawRow, idx, destRow, i, 1)
-          }
-        }
-        Row.fromSeq(destRow)
-      }
-      dictRdd
-    }
-  }
-
-  /**
-   * check whether global dictionary have been generated successfully or not
-   *
-   * @param status checking whether the generating is  successful
-   */
-  private def checkStatus(carbonLoadModel: CarbonLoadModel,
-      sqlContext: SQLContext,
-      model: DictionaryLoadModel,
-      status: Array[(Int, SegmentStatus)]) = {
-    var result = false
-    val tableName = model.table.getCarbonTableIdentifier.getTableName
-    status.foreach { x =>
-      val columnName = model.primDimensions(x._1).getColName
-      if (SegmentStatus.LOAD_FAILURE == x._2) {
-        result = true
-        LOGGER.error(s"table:$tableName column:$columnName generate global dictionary file failed")
-      }
-    }
-    if (result) {
-      LOGGER.error("generate global dictionary files failed")
-      throw new Exception("Failed to generate global dictionary files")
-    } else {
-      LOGGER.info("generate global dictionary successfully")
-    }
-  }
-
-  /**
-   * get external columns and whose dictionary file path
-   *
-   * @param colDictFilePath external column dict file path
-   * @param table           table identifier
-   * @param dimensions      dimension columns
-   */
-  private def setPredefinedColumnDictPath(carbonLoadModel: CarbonLoadModel,
-      colDictFilePath: String,
-      table: CarbonTableIdentifier,
-      dimensions: Array[CarbonDimension]) = {
-    val colFileMapArray = colDictFilePath.split(",")
-    for (colPathMap <- colFileMapArray) {
-      val colPathMapTrim = colPathMap.trim
-      val colNameWithPath = colPathMapTrim.split(":")
-      if (colNameWithPath.length == 1) {
-        LOGGER.error("the format of external column dictionary should be " +
-                     "columnName:columnPath, please check")
-        throw new DataLoadingException("the format of predefined column dictionary" +
-                                       " should be columnName:columnPath, please check")
-      }
-      setPredefineDict(carbonLoadModel, dimensions, table, colNameWithPath(0),
-        FileUtils
-          .getPaths(CarbonUtil
-            .checkAndAppendHDFSUrl(colPathMapTrim.substring(colNameWithPath(0).length + 1))))
-    }
-  }
-
-  /**
-   * set pre defined dictionary for dimension
-   *
-   * @param dimensions    all the dimensions
-   * @param table         carbon table identifier
-   * @param colName       user specified  column name for predefined dict
-   * @param colDictPath   column dictionary file path
-   * @param parentDimName parent dimension for complex type
-   */
-  def setPredefineDict(carbonLoadModel: CarbonLoadModel,
-      dimensions: Array[CarbonDimension],
-      table: CarbonTableIdentifier,
-      colName: String,
-      colDictPath: String,
-      parentDimName: String = "") {
-    val middleDimName = colName.split("\\.")(0)
-    val dimParent = parentDimName + {
-      colName match {
-        case "" => colName
-        case _ =>
-          if (parentDimName.isEmpty) {
-            middleDimName
-          } else {
-            "." + middleDimName
-          }
-      }
-    }
-    // judge whether the column is exists
-    val preDictDimensionOption = dimensions.filter(
-      _.getColName.equalsIgnoreCase(dimParent))
-    if (preDictDimensionOption.length == 0) {
-      LOGGER.error(s"Column $dimParent is not a key column " +
-                   s"in ${ table.getDatabaseName }.${ table.getTableName }")
-      throw new DataLoadingException(s"Column $dimParent is not a key column. " +
-                                     s"Only key column can be part of dictionary " +
-                                     s"and used in COLUMNDICT option.")
-    }
-    val preDictDimension = preDictDimensionOption(0)
-    if (preDictDimension.isComplex) {
-      val children = preDictDimension.getListOfChildDimensions.asScala.toArray
-      // for Array, user set ArrayFiled: path, while ArrayField has a child Array.val
-      val currentColName = {
-        if (DataTypes.isArrayType(preDictDimension.getDataType)) {
-          if (children(0).isComplex) {
-            "val." + colName.substring(middleDimName.length + 1)
-          } else {
-            "val"
-          }
-        } else {
-          colName.substring(middleDimName.length + 1)
-        }
-      }
-      setPredefineDict(carbonLoadModel, children, table, currentColName,
-        colDictPath, dimParent)
-    } else {
-      carbonLoadModel.setPredefDictMap(preDictDimension, colDictPath)
-    }
-  }
-
-  /**
-   * use external dimension column to generate global dictionary
-   *
-   * @param colDictFilePath external column dict file path
-   * @param table           table identifier
-   * @param dimensions      dimension column
-   * @param carbonLoadModel carbon load model
-   * @param sqlContext      spark sql context
-   * @param dictFolderPath  generated global dict file path
-   */
-  def generatePredefinedColDictionary(colDictFilePath: String,
-      table: CarbonTableIdentifier,
-      dimensions: Array[CarbonDimension],
-      carbonLoadModel: CarbonLoadModel,
-      sqlContext: SQLContext,
-      dictFolderPath: String): Unit = {
-    // set pre defined dictionary column
-    setPredefinedColumnDictPath(carbonLoadModel, colDictFilePath, table, dimensions)
-    val dictLoadModel = createDictionaryLoadModel(carbonLoadModel, table, dimensions,
-      dictFolderPath, forPreDefDict = true)
-    // new RDD to achieve distributed column dict generation
-    val extInputRDD = new CarbonColumnDictGenerateRDD(carbonLoadModel, dictLoadModel,
-      sqlContext.sparkSession, table, dimensions, dictFolderPath)
-      .partitionBy(new ColumnPartitioner(dictLoadModel.primDimensions.length))
-    val statusList = new CarbonGlobalDictionaryGenerateRDD(sqlContext.sparkSession, extInputRDD,
-      dictLoadModel)
-      .collect()
-    // check result status
-    checkStatus(carbonLoadModel, sqlContext, dictLoadModel, statusList)
-  }
-
-  /* generate Dimension Parsers
-   *
-   * @param model
-   * @param distinctValuesList
-   * @return dimensionParsers
-   */
-  def createDimensionParsers(model: DictionaryLoadModel,
-      distinctValuesList: ArrayBuffer[(Int, HashSet[String])]): Array[GenericParser] = {
-    // local combine set
-    val dimNum = model.dimensions.length
-    val primDimNum = model.primDimensions.length
-    val columnValues = new Array[HashSet[String]](primDimNum)
-    val mapColumnValuesWithId = new HashMap[String, HashSet[String]]
-    for (i <- 0 until primDimNum) {
-      columnValues(i) = new HashSet[String]
-      distinctValuesList += ((i, columnValues(i)))
-      mapColumnValuesWithId.put(model.primDimensions(i).getColumnId, columnValues(i))
-    }
-    val dimensionParsers = new Array[GenericParser](dimNum)
-    for (j <- 0 until dimNum) {
-      dimensionParsers(j) = GlobalDictionaryUtil.generateParserForDimension(
-        Some(model.dimensions(j)),
-        GlobalDictionaryUtil.createDataFormat(model.delimiters),
-        mapColumnValuesWithId).get
-    }
-    dimensionParsers
-  }
-
-  /**
-   * parse records in dictionary file and validate record
-   *
-   * @param x
-   * @param accum
-   * @param csvFileColumns
-   */
-  private def parseRecord(x: String, accum: Accumulator[Int],
-      csvFileColumns: Array[String]): (String, String) = {
-    val tokens = x.split("" + DEFAULT_SEPARATOR)
-    var columnName: String = ""
-    var value: String = ""
-    // such as "," , "", throw ex
-    if (tokens.isEmpty) {
-      LOGGER.error("Read a bad dictionary record: " + x)
-      accum += 1
-    } else if (tokens.size == 1) {
-      // such as "1", "jone", throw ex
-      if (!x.contains(",")) {
-        accum += 1
-      } else {
-        try {
-          columnName = csvFileColumns(tokens(0).toInt)
-        } catch {
-          case _: Exception =>
-            LOGGER.error("Read a bad dictionary record: " + x)
-            accum += 1
-        }
-      }
-    } else {
-      try {
-        columnName = csvFileColumns(tokens(0).toInt)
-        value = tokens(1)
-      } catch {
-        case _: Exception =>
-          LOGGER.error("Read a bad dictionary record: " + x)
-          accum += 1
-      }
-    }
-    (columnName, value)
-  }
-
-  /**
-   * read local dictionary and prune column
-   *
-   * @param sqlContext
-   * @param csvFileColumns
-   * @param requireColumns
-   * @param allDictionaryPath
-   * @return allDictionaryRdd
-   */
-  private def readAllDictionaryFiles(sqlContext: SQLContext,
-      csvFileColumns: Array[String],
-      requireColumns: Array[String],
-      allDictionaryPath: String,
-      accumulator: Accumulator[Int]) = {
-    var allDictionaryRdd: RDD[(String, Iterable[String])] = null
-    try {
-      // read local dictionary file, and spilt (columnIndex, columnValue)
-      val basicRdd = sqlContext.sparkContext.textFile(allDictionaryPath)
-        .map(x => parseRecord(x, accumulator, csvFileColumns))
-
-      // group by column index, and filter required columns
-      val requireColumnsList = requireColumns.toList
-      allDictionaryRdd = basicRdd
-        .groupByKey()
-        .filter(x => requireColumnsList.contains(x._1))
-    } catch {
-      case ex: Exception =>
-        LOGGER.error("Read dictionary files failed. Caused by: " + ex.getMessage)
-        throw ex
-    }
-    allDictionaryRdd
-  }
-
-  /**
-   * validate local dictionary files
-   *
-   * @param allDictionaryPath
-   * @return (isNonempty, isDirectory)
-   */
-  private def validateAllDictionaryPath(allDictionaryPath: String): Boolean = {
-    val filePath = new Path(allDictionaryPath)
-    val file = FileFactory.getCarbonFile(filePath.toString)
-    val parentFile = FileFactory.getCarbonFile(filePath.getParent.toString)
-    // filepath regex, look like "/path/*.dictionary"
-    if (filePath.getName.startsWith("*")) {
-      val dictExt = filePath.getName.substring(1)
-      if (parentFile.exists()) {
-        val listFiles = parentFile.listFiles()
-        if (listFiles.exists(file =>
-          file.getName.endsWith(dictExt) && file.getSize > 0)) {
-          true
-        } else {
-          LOGGER.warn("No dictionary files found or empty dictionary files! " +
-                      "Won't generate new dictionary.")
-          false
-        }
-      } else {
-        throw new DataLoadingException(
-          s"The given dictionary file path is not found : $allDictionaryPath")
-      }
-    } else {
-      if (file.exists()) {
-        if (file.getSize > 0) {
-          true
-        } else {
-          LOGGER.warn("No dictionary files found or empty dictionary files! " +
-                      "Won't generate new dictionary.")
-          false
-        }
-      } else {
-        throw new DataLoadingException(
-          s"The given dictionary file path is not found : $allDictionaryPath")
-      }
-    }
-  }
-
-  /**
-   * generate global dictionary with SQLContext and CarbonLoadModel
-   *
-   * @param sqlContext      sql context
-   * @param carbonLoadModel carbon load model
-   */
-  def generateGlobalDictionary(
-      sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      hadoopConf: Configuration,
-      dataFrame: Option[DataFrame] = None): Unit = {
-    try {
-      val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-      val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-      val dictfolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
-      // columns which need to generate global dictionary file
-      val dimensions = carbonTable.getVisibleDimensions().asScala.toArray
-      // generate global dict from pre defined column dict file
-      carbonLoadModel.initPredefDictMap()
-      val allDictionaryPath = carbonLoadModel.getAllDictPath
-      if (StringUtils.isEmpty(allDictionaryPath)) {
-        LOGGER.info("Generate global dictionary from source data files!")
-        // load data by using dataSource com.databricks.spark.csv
-        val headers = carbonLoadModel.getCsvHeaderColumns.map(_.trim)
-        val colDictFilePath = carbonLoadModel.getColDictFilePath
-        if (colDictFilePath != null) {
-          // generate predefined dictionary
-          generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
-            dimensions, carbonLoadModel, sqlContext, dictfolderPath)
-        }
-
-        val headerOfInputData: Array[String] = if (dataFrame.isDefined) {
-          dataFrame.get.columns
-        } else {
-          headers
-        }
-
-        if (headers.length > headerOfInputData.length && !carbonTable.isHivePartitionTable) {
-          val msg = "The number of columns in the file header do not match the " +
-                    "number of columns in the data file; Either delimiter " +
-                    "or fileheader provided is not correct"
-          LOGGER.error(msg)
-          throw new DataLoadingException(msg)
-        }
-        // use fact file to generate global dict
-        val (requireDimension, requireColumnNames) = pruneDimensions(dimensions,
-          headers, headerOfInputData)
-        if (requireDimension.nonEmpty) {
-          // select column to push down pruning
-          val dictRdd = loadInputDataAsDictRdd(sqlContext, carbonLoadModel, dataFrame,
-            requireColumnNames, hadoopConf)
-          val model = createDictionaryLoadModel(carbonLoadModel, carbonTableIdentifier,
-            requireDimension, dictfolderPath, false)
-          // combine distinct value in a block and partition by column
-          val inputRDD = new CarbonBlockDistinctValuesCombineRDD(sqlContext.sparkSession, dictRdd,
-            model)
-            .partitionBy(new ColumnPartitioner(model.primDimensions.length))
-          // generate global dictionary files
-          val statusList = new CarbonGlobalDictionaryGenerateRDD(sqlContext.sparkSession,
-            inputRDD, model)
-            .collect()
-          // check result status
-          checkStatus(carbonLoadModel, sqlContext, model, statusList)
-        } else {
-          LOGGER.info("No column found for generating global dictionary in source data files")
-        }
-      } else {
-        generateDictionaryFromDictionaryFiles(
-          sqlContext,
-          carbonLoadModel,
-          carbonTableIdentifier,
-          dictfolderPath,
-          dimensions,
-          allDictionaryPath)
-      }
-    } catch {
-      case ex: Exception =>
-        if (ex.getCause != null && ex.getCause.isInstanceOf[NoRetryException]) {
-          LOGGER.error("generate global dictionary failed", ex.getCause)
-          throw new Exception("generate global dictionary failed, " +
-                              ex.getCause.getMessage)
-        }
-        ex match {
-          case spx: SparkException =>
-            LOGGER.error("generate global dictionary failed", spx)
-            throw new Exception("generate global dictionary failed, " +
-                                trimErrorMessage(spx.getMessage))
-          case _ =>
-            LOGGER.error("generate global dictionary failed", ex)
-            throw ex
-        }
-    }
-  }
-
-  def generateDictionaryFromDictionaryFiles(
-      sqlContext: SQLContext,
-      carbonLoadModel: CarbonLoadModel,
-      carbonTableIdentifier: CarbonTableIdentifier,
-      dictFolderPath: String,
-      dimensions: Array[CarbonDimension],
-      allDictionaryPath: String): Unit = {
-    LOGGER.info("Generate global dictionary from dictionary files!")
-    val allDictionaryPathAppended = CarbonUtil.checkAndAppendHDFSUrl(allDictionaryPath)
-    val isNonempty = validateAllDictionaryPath(allDictionaryPathAppended)
-    if (isNonempty) {
-      var headers = carbonLoadModel.getCsvHeaderColumns
-      headers = headers.map(headerName => headerName.trim)
-      // prune columns according to the CSV file header, dimension columns
-      val (requireDimension, requireColumnNames) = pruneDimensions(dimensions, headers, headers)
-      if (requireDimension.nonEmpty) {
-        val model = createDictionaryLoadModel(carbonLoadModel, carbonTableIdentifier,
-          requireDimension, dictFolderPath, false)
-        // check if dictionary files contains bad record
-        val accumulator = sqlContext.sparkContext.accumulator(0)
-        // read local dictionary file, and group by key
-        val allDictionaryRdd = readAllDictionaryFiles(sqlContext, headers,
-          requireColumnNames, allDictionaryPathAppended, accumulator)
-        // read exist dictionary and combine
-        val inputRDD = new CarbonAllDictionaryCombineRDD(sqlContext.sparkSession,
-          allDictionaryRdd, model)
-          .partitionBy(new ColumnPartitioner(model.primDimensions.length))
-        // generate global dictionary files
-        val statusList = new CarbonGlobalDictionaryGenerateRDD(sqlContext.sparkSession, inputRDD,
-          model)
-          .collect()
-        // check result status
-        checkStatus(carbonLoadModel, sqlContext, model, statusList)
-        // if the dictionary contains wrong format record, throw ex
-        if (accumulator.value > 0) {
-          throw new DataLoadingException("Data Loading failure, dictionary values are " +
-                                         "not in correct format!")
-        }
-      } else {
-        LOGGER.info("have no column need to generate global dictionary")
-      }
-    }
-  }
-
-  // Get proper error message of TextParsingException
-  def trimErrorMessage(input: String): String = {
-    var errorMessage: String = null
-    if (input != null && input.contains("TextParsingException:")) {
-      if (input.split("Hint").length > 1 &&
-          input.split("Hint")(0).split("TextParsingException: ").length > 1) {
-        errorMessage = input.split("Hint")(0).split("TextParsingException: ")(1)
-      } else if (input.split("Parser Configuration:").length > 1) {
-        errorMessage = input.split("Parser Configuration:")(0)
-      }
-    } else if (input != null && input.contains("Exception:")) {
-      errorMessage = input.split("Exception: ")(1).split("\n")(0)
-    }
-    errorMessage
-  }
-
-  /**
-   * This method will write dictionary file, sortindex file and dictionary meta for new dictionary
-   * column with default value
-   *
-   * @param columnSchema
-   * @param absoluteTableIdentifier
-   * @param defaultValue
-   */
-  def loadDefaultDictionaryValueForNewColumn(
-      columnSchema: ColumnSchema,
-      absoluteTableIdentifier: AbsoluteTableIdentifier,
-      defaultValue: String): Unit = {
-
-    val dictLock = CarbonLockFactory
-      .getCarbonLockObj(absoluteTableIdentifier,
-        columnSchema.getColumnUniqueId + LockUsage.LOCK)
-    var isDictionaryLocked = false
-    try {
-      isDictionaryLocked = dictLock.lockWithRetries()
-      if (isDictionaryLocked) {
-        LOGGER.info(s"Successfully able to get the dictionary lock for ${
-          columnSchema.getColumnName
-        }")
-      } else {
-        sys.error(s"Dictionary file ${
-          columnSchema.getColumnName
-        } is locked for updation. Please try after some time")
-      }
-      val columnIdentifier = new ColumnIdentifier(columnSchema.getColumnUniqueId,
-        null,
-        columnSchema.getDataType)
-      val dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier = new
-          DictionaryColumnUniqueIdentifier(
-            absoluteTableIdentifier,
-            columnIdentifier,
-            columnIdentifier.getDataType)
-      val parsedValue = DataTypeUtil.normalizeColumnValueForItsDataType(defaultValue, columnSchema)
-      val valuesBuffer = new mutable.HashSet[String]
-      if (null != parsedValue) {
-        valuesBuffer += parsedValue
-      }
-      val dictWriteTask = new DictionaryWriterTask(valuesBuffer,
-        dictionary = null,
-        dictionaryColumnUniqueIdentifier,
-        columnSchema,
-        false
-      )
-      val distinctValues = dictWriteTask.execute
-      LOGGER.info(s"Dictionary file writing is successful for new column ${
-        columnSchema.getColumnName
-      }")
-
-      if (distinctValues.size() > 0) {
-        val sortIndexWriteTask = new SortIndexWriterTask(
-          dictionaryColumnUniqueIdentifier,
-          columnSchema.getDataType,
-          dictionary = null,
-          distinctValues)
-        sortIndexWriteTask.execute()
-      }
-
-      LOGGER.info(s"SortIndex file writing is successful for new column ${
-        columnSchema.getColumnName
-      }")
-
-      // After sortIndex writing, update dictionaryMeta
-      dictWriteTask.updateMetaData()
-
-      LOGGER.info(s"Dictionary meta file writing is successful for new column ${
-        columnSchema.getColumnName
-      }")
-    } catch {
-      case ex: Exception =>
-        LOGGER.error(ex)
-        throw ex
-    } finally {
-      if (dictLock != null && isDictionaryLocked) {
-        if (dictLock.unlock()) {
-          LOGGER.info(s"Dictionary ${
-            columnSchema.getColumnName
-          } Unlocked Successfully.")
-        } else {
-          LOGGER.error(s"Unable to unlock Dictionary ${
-            columnSchema.getColumnName
-          }")
-        }
-      }
-    }
-  }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 0f25703..3c55f78 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -32,8 +32,6 @@
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.{DictionaryServer, NonSecureDictionaryServer}
-import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceProvider
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
@@ -43,8 +41,6 @@
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent, LoadTablePreStatusUpdateEvent}
 import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.processing.util.CarbonBadRecordUtil
-import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
-import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
 import org.apache.carbondata.streaming.segment.StreamSegment
 
 /**
@@ -111,16 +107,6 @@
     // Used to generate load commands for child tables in case auto-handoff is fired.
     val loadMetaEvent = new LoadMetadataEvent(carbonTable, false, parameters.asJava)
     OperationListenerBus.getInstance().fireEvent(loadMetaEvent, operationContext)
-    // start server if necessary
-    val server = startDictionaryServer(
-      sparkSession,
-      carbonTable,
-      carbonLoadModel)
-    if (server.isDefined) {
-      carbonLoadModel.setUseOnePass(true)
-    } else {
-      carbonLoadModel.setUseOnePass(false)
-    }
 
     // default is carbon appended stream sink
     val carbonAppendableStreamSink = new CarbonAppendableStreamSink(
@@ -129,7 +115,6 @@
       segmentId,
       parameters,
       carbonLoadModel,
-      server,
       operationContext)
 
     // fire post event before streamin is started
@@ -181,63 +166,6 @@
     segmentId
   }
 
-  def startDictionaryServer(
-      sparkSession: SparkSession,
-      carbonTable: CarbonTable,
-      carbonLoadModel: CarbonLoadModel): Option[DictionaryServer] = {
-    // start dictionary server when use one pass load and dimension with DICTIONARY
-    // encoding is present.
-    val allDimensions = carbonTable.getAllDimensions.asScala.toList
-    val createDictionary = allDimensions.exists {
-      carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-                         !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
-    }
-    val carbonSecureModeDictServer = CarbonProperties.getInstance.
-      getProperty(CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER,
-        CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER_DEFAULT)
-
-    val sparkConf = sparkSession.sqlContext.sparkContext.getConf
-    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-      getConf.get("spark.driver.host")
-
-    val server: Option[DictionaryServer] = if (createDictionary) {
-      if (sparkConf.get("spark.authenticate", "false").equalsIgnoreCase("true") &&
-          carbonSecureModeDictServer.toBoolean) {
-        val dictionaryServer = SecureDictionaryServer.getInstance(sparkConf,
-          sparkDriverHost.toString, carbonLoadModel.getDictionaryServerPort, carbonTable)
-        carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-        carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
-        carbonLoadModel.setDictionaryServerSecretKey(dictionaryServer.getSecretKey)
-        carbonLoadModel.setDictionaryEncryptServerSecure(dictionaryServer.isEncryptSecureServer)
-        carbonLoadModel.setDictionaryServiceProvider(new SecureDictionaryServiceProvider())
-        sparkSession.sparkContext.addSparkListener(new SparkListener() {
-          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-            dictionaryServer.shutdown()
-          }
-        })
-        Some(dictionaryServer)
-      } else {
-        val dictionaryServer = NonSecureDictionaryServer
-          .getInstance(carbonLoadModel.getDictionaryServerPort, carbonTable)
-        carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-        carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
-        carbonLoadModel.setDictionaryEncryptServerSecure(false)
-        carbonLoadModel
-          .setDictionaryServiceProvider(new NonSecureDictionaryServiceProvider(dictionaryServer
-            .getPort))
-        sparkSession.sparkContext.addSparkListener(new SparkListener() {
-          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-            dictionaryServer.shutdown()
-          }
-        })
-        Some(dictionaryServer)
-      }
-    } else {
-      None
-    }
-    server
-  }
-
   private def buildCarbonLoadModelForStream(
       sparkSession: SparkSession,
       hadoopConf: Configuration,
@@ -261,16 +189,6 @@
       carbonLoadModel,
       hadoopConf)
     carbonLoadModel.setSegmentId(segmentId)
-    // stream should use one pass
-    val dictionaryServerPort = parameters.getOrElse(
-      CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-      carbonProperty.getProperty(
-        CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-        CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT))
-    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-      getConf.get("spark.driver.host")
-    carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
-    carbonLoadModel.setDictionaryServerPort(dictionaryServerPort.toInt)
     val columnCompressor = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
       .getOrElse(CarbonCommonConstants.COMPRESSOR,
         CompressorFactory.getInstance().getCompressor.getName)
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
index dfbcf07..41e8397 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadPartitionCoalescer.scala
@@ -196,6 +196,7 @@
                   case None =>
                 }
               }
+            case None =>
           }
         } else {
           noEmptyHostSet.remove(hostEntry._1)
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 51c8cfd..215439a 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -32,6 +32,7 @@
 import org.apache.spark.util.PartitionUtils
 
 import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -412,8 +413,6 @@
       }
     }
 
-    // get partitionInfo
-    val partitionInfo = getPartitionInfo(partitionCols)
     if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
       // validate the column_meta_cache option
       val tableColumns = dims.map(x => x.name.get) ++ msrs.map(x => x.name.get)
@@ -430,7 +429,7 @@
           // first element is taken as each column with have a unique name
           // check for complex type column
           if (dimFieldToBeCached.nonEmpty &&
-              isComplexDimDictionaryExclude(dimFieldToBeCached(0).dataType.get)) {
+              isComplexType(dimFieldToBeCached(0).dataType.get)) {
             val errorMessage =
               s"$column is a complex type column and complex type is not allowed for " +
               s"the option(s): ${ CarbonCommonConstants.COLUMN_META_CACHE }"
@@ -496,7 +495,7 @@
       Option(invertedIdxCols),
       Some(colProps),
       bucketFields: Option[BucketFields],
-      partitionInfo,
+      getPartitionInfo(partitionCols),
       tableComment)
   }
 
@@ -702,9 +701,7 @@
   (Seq[Field], Seq[Field], Seq[String], Seq[String], Seq[String]) = {
     var dimFields: LinkedHashSet[Field] = LinkedHashSet[Field]()
     var msrFields: Seq[Field] = Seq[Field]()
-    var dictExcludeCols: Array[String] = Array[String]()
     var noDictionaryDims: Seq[String] = Seq[String]()
-    var dictIncludeCols: Seq[String] = Seq[String]()
     var varcharCols: Seq[String] = Seq[String]()
 
     // All long_string cols should be there in create table cols and should be of string data type
@@ -767,78 +764,19 @@
       }
     }
 
-    // All excluded cols should be there in create table cols
     if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
-      LOGGER.warn("dictionary_exclude option was deprecated, " +
-                  "by default string column does not use global dictionary.")
-      dictExcludeCols =
-        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
-      dictExcludeCols
-        .foreach { dictExcludeCol =>
-          if (!fields.exists(x => x.column.equalsIgnoreCase(dictExcludeCol))) {
-            val errorMsg = "DICTIONARY_EXCLUDE column: " + dictExcludeCol +
-                           " does not exist in table or unsupported for complex child column. " +
-                           "Please check the create table statement."
-            throw new MalformedCarbonCommandException(errorMsg)
-          } else {
-            val dataType = fields.find(x =>
-              x.column.equalsIgnoreCase(dictExcludeCol)).get.dataType.get
-            if (!isDataTypeSupportedForDictionary_Exclude(dataType)) {
-              val errorMsg = "DICTIONARY_EXCLUDE is unsupported for " + dataType.toLowerCase() +
-                             " data type column: " + dictExcludeCol
-              throw new MalformedCarbonCommandException(errorMsg)
-            } else if (varcharCols.exists(x => x.equalsIgnoreCase(dictExcludeCol))) {
-              throw new MalformedCarbonCommandException(
-                "DICTIONARY_EXCLUDE is unsupported for long string datatype column: " +
-                dictExcludeCol)
-            }
-          }
-        }
+      // dictionary_exclude is not supported since 2.0
+      throw new DeprecatedFeatureException("dictionary_exclude")
     }
-    // All included cols should be there in create table cols
     if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
-      dictIncludeCols =
-        tableProperties(CarbonCommonConstants.DICTIONARY_INCLUDE).split(",").map(_.trim)
-      dictIncludeCols.foreach { distIncludeCol =>
-        if (!fields.exists(x => x.column.equalsIgnoreCase(distIncludeCol.trim))) {
-          val errorMsg = "DICTIONARY_INCLUDE column: " + distIncludeCol.trim +
-                         " does not exist in table or unsupported for complex child column. " +
-                         "Please check the create table statement."
-          throw new MalformedCarbonCommandException(errorMsg)
-        }
-        val rangeField = fields.find(_.column.equalsIgnoreCase(distIncludeCol.trim))
-        if ("binary".equalsIgnoreCase(rangeField.get.dataType.get)) {
-          throw new MalformedCarbonCommandException(
-            "DICTIONARY_INCLUDE is unsupported for binary data type column: " +
-                    distIncludeCol.trim)
-        }
-        if (varcharCols.exists(x => x.equalsIgnoreCase(distIncludeCol.trim))) {
-          throw new MalformedCarbonCommandException(
-            "DICTIONARY_INCLUDE is unsupported for long string datatype column: " +
-            distIncludeCol.trim)
-        }
-      }
-    }
-
-    // include cols should not contain exclude cols
-    dictExcludeCols.foreach { dicExcludeCol =>
-      if (dictIncludeCols.exists(x => x.equalsIgnoreCase(dicExcludeCol))) {
-        val errorMsg = "DICTIONARY_EXCLUDE can not contain the same column: " + dicExcludeCol +
-                       " with DICTIONARY_INCLUDE. Please check the create table statement."
-        throw new MalformedCarbonCommandException(errorMsg)
-      }
+      // dictionary_include is not supported since 2.0
+      throw new DeprecatedFeatureException("dictionary_include")
     }
 
     // by default consider all String cols as dims and if any dictionary include isn't present then
     // add it to noDictionaryDims list. consider all dictionary excludes/include cols as dims
     fields.foreach { field =>
-      if (dictExcludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
-        noDictionaryDims :+= field.column
-        dimFields += field
-      } else if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
-        dimFields += field
-      } else if (field.dataType.get.toUpperCase.equals("TIMESTAMP") &&
-                 !dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
+      if (field.dataType.get.toUpperCase.equals("TIMESTAMP")) {
         noDictionaryDims :+= field.column
         dimFields += field
       } else if (isDetectAsDimensionDataType(field.dataType.get)) {
@@ -867,7 +805,7 @@
       // if SORT_COLUMNS was not defined,
       // add all dimension(except long string columns) to SORT_COLUMNS.
       dimFields.foreach { field =>
-        if (!isComplexDimDictionaryExclude(field.dataType.get) &&
+        if (!isComplexType(field.dataType.get) &&
             !varcharCols.contains(field.column)) {
           sortKeyDims :+= field.column
         }
@@ -888,29 +826,13 @@
   }
 
   /**
-   * It fills non string dimensions in dimFields
-   */
-  def fillNonStringDimension(dictIncludeCols: Seq[String],
-      field: Field, dimFields: LinkedHashSet[Field]) {
-    var dictInclude = false
-    if (dictIncludeCols.nonEmpty) {
-      dictIncludeCols.foreach(dictIncludeCol =>
-        if (field.column.equalsIgnoreCase(dictIncludeCol)) {
-          dictInclude = true
-        })
-    }
-    if (dictInclude) {
-      dimFields += field
-    }
-  }
-
-  /**
    * detect dimension data type
    *
    * @param dimensionDatatype
    */
   def isDetectAsDimensionDataType(dimensionDatatype: String): Boolean = {
-    val dimensionType = Array("string",
+    val dimensionType = Array(
+      "string",
       "array",
       "struct",
       "map",
@@ -922,32 +844,14 @@
   }
 
   /**
-   * detects whether complex dimension is part of dictionary_exclude
+   * detects whether it is complex type
    */
-  def isComplexDimDictionaryExclude(dimensionDataType: String): Boolean = {
+  def isComplexType(dimensionDataType: String): Boolean = {
     val dimensionType = Array("array", "struct", "map")
     dimensionType.exists(x => x.equalsIgnoreCase(dimensionDataType))
   }
 
   /**
-   * detects whether datatype is part of sort_column
-   */
-  private def isDataTypeSupportedForSortColumn(columnDataType: String): Boolean = {
-    val dataTypes = Array("array", "struct", "map", "double", "float", "decimal", "binary")
-    dataTypes.exists(x => x.equalsIgnoreCase(columnDataType))
-  }
-
-  /**
-   * detects whether datatype is part of dictionary_exclude
-   */
-  def isDataTypeSupportedForDictionary_Exclude(columnDataType: String): Boolean = {
-    val dataTypes =
-      Array("string", "timestamp", "int", "integer", "long", "bigint", "struct", "array",
-        "map", "binary")
-    dataTypes.exists(x => x.equalsIgnoreCase(columnDataType))
-  }
-
-  /**
    * Extract the DbName and table name.
    *
    * @param tableNameParts
@@ -1072,17 +976,14 @@
       "COMPLEX_DELIMITER_LEVEL_1",
       "COMPLEX_DELIMITER_LEVEL_2",
       "COMPLEX_DELIMITER_LEVEL_3",
-      "COLUMNDICT",
       "SERIALIZATION_NULL_FORMAT",
       "BAD_RECORDS_LOGGER_ENABLE",
       "BAD_RECORDS_ACTION",
-      "ALL_DICTIONARY_PATH",
       "MAXCOLUMNS",
       "COMMENTCHAR",
       "DATEFORMAT",
       "BAD_RECORD_PATH",
       "GLOBAL_SORT_PARTITIONS",
-      "SINGLE_PASS",
       "IS_EMPTY_DATA_BAD_RECORD",
       "HEADER",
       "TIMESTAMPFORMAT",
@@ -1132,14 +1033,6 @@
       }
     }
 
-    //  COLUMNDICT and ALL_DICTIONARY_PATH can not be used together.
-    if (options.exists(_._1.equalsIgnoreCase("COLUMNDICT")) &&
-        options.exists(_._1.equalsIgnoreCase("ALL_DICTIONARY_PATH"))) {
-      val errorMessage = "Error: COLUMNDICT and ALL_DICTIONARY_PATH can not be used together" +
-                         " in options"
-      throw new MalformedCarbonCommandException(errorMessage)
-    }
-
     if (options.exists(_._1.equalsIgnoreCase("MAXCOLUMNS"))) {
       val maxColumns: String = options("maxcolumns").head._2
       try {
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index b6441fb..523ec9e 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -257,7 +257,6 @@
     // add new dimension columns
     alterTableModel.dimCols.foreach(field => {
       val encoders = new java.util.ArrayList[Encoding]()
-      encoders.add(Encoding.DICTIONARY)
       val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
         field,
         encoders,
@@ -570,9 +569,10 @@
       encoders.remove(Encoding.DICTIONARY)
     }
     if (dataType == DataTypes.DATE) {
+      encoders.add(Encoding.DICTIONARY)
       encoders.add(Encoding.DIRECT_DICTIONARY)
-    }
-    if (dataType == DataTypes.TIMESTAMP && !highCardinalityDims.contains(colName)) {
+    } else if (dataType == DataTypes.TIMESTAMP && !highCardinalityDims.contains(colName)) {
+      encoders.add(Encoding.DICTIONARY)
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
     columnSchema.setEncodingList(encoders)
@@ -597,9 +597,6 @@
     fieldChildren.foreach(fields => {
       fields.foreach(field => {
         val encoders = new java.util.ArrayList[Encoding]()
-        if (useDictionaryEncoding) {
-          encoders.add(Encoding.DICTIONARY)
-        }
         val columnSchema: ColumnSchema = getColumnSchema(
           DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
           field.name.getOrElse(field.column),
@@ -727,15 +724,13 @@
       index = index + 1
     }
 
-    val dictionaryIncludeCols = cm.tableProperties
-      .getOrElse(CarbonCommonConstants.DICTIONARY_INCLUDE, "")
-
     def addDimensionCol(field: Field): Unit = {
       val sortField = cm.sortKeyDims.get.find(field.column equals _)
       if (sortField.isEmpty) {
         val encoders = if (getEncoderFromParent(field)) {
-          cm.parentTable.get.getColumnByName(cm.dataMapRelation.get.get(field).get.
-                        columnTableRelationList.get(0).parentColumnName).getEncoder
+          cm.parentTable.get.getColumnByName(
+            cm.dataMapRelation.get(field).columnTableRelationList.get.head.parentColumnName
+          ).getEncoder
         } else {
           val encoders = new java.util.ArrayList[Encoding]()
           encoders.add(Encoding.DICTIONARY)
@@ -751,17 +746,14 @@
         allColumns :+= columnSchema
         index = index + 1
         if (field.children.isDefined && field.children.get != null) {
-          val includeDictionaryEncoding = dictionaryIncludeCols.contains(field.column)
-          if (!includeDictionaryEncoding) {
-            columnSchema.getEncodingList.remove(Encoding.DICTIONARY)
-          }
+          columnSchema.getEncodingList.remove(Encoding.DICTIONARY)
           columnSchema.setNumberOfChild(field.children.get.size)
-          allColumns ++= getAllChildren(field.children, includeDictionaryEncoding)
+          allColumns ++= getAllChildren(field.children, false)
         }
       }
     }
     // add all dimensions
-    cm.dimCols.foreach(addDimensionCol(_))
+    cm.dimCols.foreach(addDimensionCol)
 
     // check whether the column is a local dictionary column and set in column schema
     if (null != cm.tableProperties) {
@@ -816,14 +808,11 @@
       }
     }
 
-    val highCardinalityDims = cm.highcardinalitydims.getOrElse(Seq())
-
     // Setting the boolean value of useInvertedIndex in column schema, if Parent table is defined
     // Encoding is already decided above
-    if (!cm.parentTable.isDefined) {
+    if (cm.parentTable.isEmpty) {
       val noInvertedIndexCols = cm.noInvertedIdxCols.getOrElse(Seq())
       val invertedIndexCols = cm.innvertedIdxCols.getOrElse(Seq())
-      LOGGER.info("NoINVERTEDINDEX columns are : " + noInvertedIndexCols.mkString(","))
       for (column <- allColumns) {
         // When the column is measure or the specified no inverted index column in DDL,
         // set useInvertedIndex to false, otherwise true.
@@ -915,8 +904,8 @@
     tableSchema.setTableName(cm.tableName)
     tableSchema.setListOfColumns(allColumns.asJava)
     tableSchema.setSchemaEvolution(schemaEvol)
-    tableInfo.setDatabaseName(cm.databaseNameOp.getOrElse(null))
-    tableInfo.setTableUniqueName(CarbonTable.buildUniqueName(cm.databaseNameOp.getOrElse(null),
+    tableInfo.setDatabaseName(cm.databaseNameOp.orNull)
+    tableInfo.setTableUniqueName(CarbonTable.buildUniqueName(cm.databaseNameOp.orNull,
       cm.tableName))
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
@@ -946,7 +935,7 @@
   private def getEncoderFromParent(field: Field): Boolean = {
     cm.parentTable.isDefined &&
     cm.dataMapRelation.get.get(field).isDefined &&
-    cm.dataMapRelation.get.get(field).get.columnTableRelationList.size == 1
+    cm.dataMapRelation.get(field).columnTableRelationList.size == 1
   }
 
   //  For checking if the specified col group columns are specified in fields list.
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 6384bf0..79f6192 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -38,7 +38,6 @@
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.metadata.datatype.DataType
 import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
@@ -67,7 +66,6 @@
     var currentSegmentId: String,
     parameters: Map[String, String],
     carbonLoadModel: CarbonLoadModel,
-    server: Option[DictionaryServer],
     operationContext: OperationContext) extends Sink {
 
   private val fileLogPath = CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)
@@ -156,7 +154,6 @@
         committer,
         hadoopConf,
         carbonLoadModel,
-        server,
         msrDataTypes)
       // fire post event on every batch add
       val loadTablePostExecutionEvent = new LoadTablePostExecutionEvent(
@@ -233,7 +230,6 @@
       committer: FileCommitProtocol,
       hadoopConf: Configuration,
       carbonLoadModel: CarbonLoadModel,
-      server: Option[DictionaryServer],
       msrDataTypes: Array[DataType]): Unit = {
 
     // create job
@@ -254,10 +250,6 @@
       var result: Array[(TaskCommitMessage, StreamFileIndex)] = null
       try {
         committer.setupJob(job)
-        // initialize dictionary server
-        if (server.isDefined) {
-          server.get.initializeDictionaryGenerator(carbonTable)
-        }
 
         val rowSchema = queryExecution.analyzed.schema
         val isVarcharTypeMapping = {
@@ -285,19 +277,6 @@
             )
           })
 
-        // write dictionary
-        if (server.isDefined) {
-          try {
-            server.get.writeTableDictionary(carbonTable.getCarbonTableIdentifier.getTableId)
-          } catch {
-            case _: Exception =>
-              LOGGER.error(
-                s"Error while writing dictionary file for ${carbonTable.getTableUniqueName}")
-              throw new Exception(
-                "Streaming ingest failed due to error while writing dictionary file")
-          }
-        }
-
         // update data file info in index file
         StreamSegment.updateIndexFile(
           CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId),
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/profiler/ProfilerLogger.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/profiler/ProfilerLogger.scala
index d474090..438365e 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/profiler/ProfilerLogger.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/profiler/ProfilerLogger.scala
@@ -133,8 +133,7 @@
    * | charField CHAR(5),                                        |
    * | floatField FLOAT                                          |
    * | )                                                         |
-   * | STORED BY 'carbondata'                                    |
-   * | TBLPROPERTIES('DICTIONARY_INCLUDE'='dateField, charField')|
+   * | STORED AS carbondata                                      |
    * +-----------------------------------------------------------+
    * [start time]: 2018-03-22 17:12:18.310
    * [driver side total taken]: 1255 ms
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
index f70d7cb..e7bb4be 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
@@ -175,8 +175,6 @@
   CarbonProperties.getInstance()
     .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FORCE")
     .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC, badStoreLocation)
-    .addProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-      (CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT.toInt + Random.nextInt(100)) + "")
     .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "1024")
     .addProperty(CarbonCommonConstants.CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE, "1024")
     .addProperty(CarbonCommonConstants.CARBON_SYSTEM_FOLDER_LOCATION, systemFolderPath)
diff --git a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index a1223ff..a50dca1 100644
--- a/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark-datasource/src/main/scala/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -25,7 +25,6 @@
 
 import org.apache.log4j.Logger;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -167,13 +166,6 @@
       vectorProxy.close();
       vectorProxy = null;
     }
-    // clear dictionary cache
-    Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
-    if (null != columnToDictionaryMapping) {
-      for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
-        CarbonUtil.clearDictionaryCache(entry.getValue());
-      }
-    }
     try {
       queryExecutor.finish();
     } catch (QueryExecutionException e) {
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceBinaryTest.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceBinaryTest.scala
index b1c2832..c5aae8d 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceBinaryTest.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceBinaryTest.scala
@@ -469,19 +469,19 @@
              """.stripMargin)
         sql("insert into carbon_table select * from hivetable")
 
-        def getHexString(str: String) = {
-            str.toList.map(_.toInt.toHexString).mkString
-        }
         sqlContext.udf.register("decodeHex", (str: String) =>
-            Hex.decodeHex(getHexString(str).toCharArray))
+            Hex.decodeHex(str.toList.map(_.toInt.toBinaryString).mkString.toCharArray))
+        sqlContext.udf.register("unHexValue", (str: String) =>
+            org.apache.spark.sql.catalyst.expressions.Hex.unhex(str.toList.map(_.toInt.toBinaryString).mkString.getBytes))
         sqlContext.udf.register("decodeBase64", (str: String) => Base64.decodeBase64(str.getBytes()))
 
         val udfHexResult = sql("SELECT decodeHex(image) FROM carbon_table")
+        val unHexResult = sql("SELECT unHexValue(image) FROM carbon_table")
+        checkAnswer(udfHexResult, unHexResult)
 
         val udfBase64Result = sql("SELECT decodeBase64(image) FROM carbon_table")
         val unbase64Result = sql("SELECT unbase64(image) FROM carbon_table")
         checkAnswer(udfBase64Result, unbase64Result)
-        checkAnswer(udfHexResult, unbase64Result)
 
         val carbonResult = sql("SELECT * FROM carbon_table")
         val hiveResult = sql("SELECT * FROM hivetable")
@@ -496,7 +496,7 @@
                 assert("\u0001education\u0002".equals(new String(each.getAs[Array[Byte]](3))))
             } else if (3 == each.get(0)) {
                 assert("".equals(new String(each.getAs[Array[Byte]](3)))
-                        || "\u0001biology\u0002".equals(new String(each.getAs[Array[Byte]](3))))
+                       || "\u0001biology\u0002".equals(new String(each.getAs[Array[Byte]](3))))
             } else {
                 assert(false)
             }
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
index 10772b1..bd42c13 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
@@ -26,14 +26,16 @@
 import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
 import org.apache.spark.sql.{AnalysisException, Row}
 import org.apache.spark.sql.carbondata.datasource.TestUtil._
-import org.apache.spark.sql.types.{BinaryType, IntegerType, StringType, StructField => SparkStructField, StructType}
+import org.apache.spark.sql.types.{BinaryType, IntegerType, StringType, StructType, StructField => SparkStructField}
 import org.apache.spark.util.SparkUtil
 import org.scalatest.{BeforeAndAfterAll, FunSuite}
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.{DataTypes, StructField}
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.hadoop.testutil.StoreCreator
 import org.apache.carbondata.sdk.file.{CarbonWriter, Field, Schema}
 
@@ -1233,8 +1235,7 @@
 
   test("test read using old data") {
     val store = new StoreCreator(new File(warehouse1).getAbsolutePath,
-      new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath,
-      false)
+      new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath)
     store.createCarbonStore()
     FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb/testtable/Fact/Part0/Segment_0/0"))
     val dfread = spark.read.format("carbon").load(warehouse1+"/testdb/testtable/Fact/Part0/Segment_0")
@@ -1243,12 +1244,12 @@
   }
 
   test("test read using different sort order data") {
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_WRITTEN_BY_APPNAME, "test")
     if (!spark.sparkContext.version.startsWith("2.1")) {
       spark.sql("drop table if exists old_comp")
       FileFactory.deleteAllFilesOfDir(new File(warehouse1 + "/testdb"))
       val store = new StoreCreator(new File(warehouse1).getAbsolutePath,
-        new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath,
-        false)
+        new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath)
       store.setSortColumns(new util.ArrayList[String](Seq("name").asJava))
       var model = store.createTableAndLoadModel(false)
       model.setSegmentId("0")
@@ -1451,7 +1452,7 @@
       }
     } catch {
       case ex: Exception => throw new RuntimeException(ex)
-      case _ => None
+      case _: Throwable => None
     }
     checkAnswer(spark.sql("select * from complextable limit 1"), Seq(Row("name0", Row(0
       .asInstanceOf[Byte], 0.012.asInstanceOf[Float]))))
@@ -1536,7 +1537,7 @@
         spark.sql("select * from sort_table"))
     } catch {
       case ex: Exception => throw new RuntimeException(ex)
-      case _ => None
+      case _: Throwable => None
     }
   }
 
@@ -1583,7 +1584,7 @@
       }
     } catch {
       case ex: Exception => throw new RuntimeException(ex)
-      case _ => None
+      case _: Throwable => None
     }
     checkAnswer(spark.sql("select * from complextable limit 1"), Seq(Row("name0", mutable
       .WrappedArray.make(Array[Byte](0, 0)), mutable.WrappedArray.make(Array[Float](0.0f, 0.0f)))))
@@ -1671,7 +1672,7 @@
       writer.close()
     } catch {
       case ex: Exception => throw new RuntimeException(ex)
-      case _ => None
+      case _: Throwable => None
     }
   }
 
@@ -1945,22 +1946,19 @@
     }
 
     test("test spark doesn't support input string value for binary data type") {
+        val rdd = spark.sparkContext.parallelize(1 to 3)
+                .map(x => Row("a" + x % 10, "b", x, "YWJj".getBytes()))
+        val customSchema = StructType(Array(
+            SparkStructField("c1", StringType),
+            SparkStructField("c2", StringType),
+            SparkStructField("number", IntegerType),
+            SparkStructField("c4", BinaryType)))
+
         try {
-            val rdd = spark.sparkContext.parallelize(1 to 3)
-                    .map(x => Row("a" + x % 10, "b", x, "YWJj".getBytes()))
-            val customSchema = StructType(Array(
-                SparkStructField("c1", StringType),
-                SparkStructField("c2", StringType),
-                SparkStructField("number", IntegerType),
-                SparkStructField("c4", BinaryType)))
-
-            try {
-                spark.createDataFrame(rdd, customSchema);
-            } catch {
-                case e: RuntimeException => e.getMessage.contains(
-                    "java.lang.String is not a valid external type for schema of binary")
-            }
-
+            spark.createDataFrame(rdd, customSchema);
+        } catch {
+            case e: RuntimeException => e.getMessage.contains(
+                "java.lang.String is not a valid external type for schema of binary")
         }
     }
 
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
index 672e972..7b39391 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/TestUtil.scala
@@ -110,18 +110,7 @@
       }
       if (!isSorted) converted.sortBy(_.toString()) else converted
     }
-    val sparkAnswer = try df.collect().toSeq catch {
-      case e: Exception =>
-        val errorMessage =
-          s"""
-             |Exception thrown while executing query:
-             |${df.queryExecution}
-             |== Exception ==
-             |$e
-             |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)}
-          """.stripMargin
-        return Some(errorMessage)
-    }
+    val sparkAnswer = df.collect().toSeq
 
     if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) {
       val errorMessage =
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala
index 14231ed..51f7dff 100644
--- a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSqlConf.scala
@@ -72,11 +72,6 @@
         .stringConf
         .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
           CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    val SINGLE_PASS =
-      buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS)
-        .doc("Property to enable/disable single_pass.")
-        .booleanConf
-        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
     val BAD_RECORD_PATH =
       buildConf(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH)
         .doc("Property to configure the bad record location.")
@@ -122,8 +117,6 @@
     sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
       carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
         CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-      CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
     sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
       carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
         CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 93101e9..f385606 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -50,7 +50,6 @@
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion, SegmentFileStore}
@@ -307,7 +306,6 @@
       carbonLoadModel: CarbonLoadModel,
       columnar: Boolean,
       partitionStatus: SegmentStatus = SegmentStatus.SUCCESS,
-      result: Option[DictionaryServer],
       overwriteTable: Boolean,
       hadoopConf: Configuration,
       dataFrame: Option[DataFrame] = None,
@@ -442,8 +440,6 @@
       } else {
         // in success case handle updation of the table status file.
         // success case.
-        // write the dictionary file in case of single_pass true
-        writeDictionary(carbonLoadModel, result, false)
         val segmentDetails = new util.HashSet[Segment]()
         var resultSize = 0
         res.foreach { resultOfSeg =>
@@ -518,8 +514,6 @@
           loadStatus
         }
 
-      writeDictionary(carbonLoadModel, result, writeAll = false)
-
       val segmentFileName =
         SegmentFileStore.writeSegmentFile(carbonTable, carbonLoadModel.getSegmentId,
           String.valueOf(carbonLoadModel.getFactTimeStamp))
@@ -810,28 +804,6 @@
   }
 
   /**
-   * Trigger to write dictionary files
-   */
-  private def writeDictionary(carbonLoadModel: CarbonLoadModel,
-      result: Option[DictionaryServer], writeAll: Boolean): Unit = {
-    // write dictionary file
-    val uniqueTableName: String =
-      CarbonTable.buildUniqueName(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
-    result match {
-      case Some(server) =>
-        try {
-          server.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-            .getCarbonTableIdentifier.getTableId)
-        } catch {
-          case _: Exception =>
-            LOGGER.error(s"Error while writing dictionary file for $uniqueTableName")
-            throw new Exception("Dataload failed due to error while writing dictionary file!")
-        }
-      case _ =>
-    }
-  }
-
-  /**
    * Trigger compaction after data load
    */
   def handleSegmentMerging(
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
index 6e7daaa..ad4cd8c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -83,8 +83,6 @@
     val property = Map(
       "SORT_COLUMNS" -> options.sortColumns,
       "SORT_SCOPE" -> options.sortScope,
-      "DICTIONARY_INCLUDE" -> options.dictionaryInclude,
-      "DICTIONARY_EXCLUDE" -> options.dictionaryExclude,
       "LONG_STRING_COLUMNS" -> options.longStringColumns,
       "TABLE_BLOCKSIZE" -> options.tableBlockSize,
       "TABLE_BLOCKLET_SIZE" -> options.tableBlockletSize,
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 6d833b9..e020a99 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
@@ -333,11 +333,8 @@
     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).getAbsoluteTableIdentifier,
-            f._2, f._3.getDataType, dictionaryPath))
+            atiMap(f._1).getAbsoluteTableIdentifier, f._2, f._3.getDataType))
         } catch {
           case _: Throwable => null
         }
@@ -369,12 +366,9 @@
               } else {
                 (atiMap(tableName).getAbsoluteTableIdentifier, columnIdentifier)
               }
-            val dictionaryPath = atiMap(tableName).getTableInfo.getFactTable.getTableProperties
-              .get(CarbonCommonConstants.DICTIONARY_PATH)
             val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
               newAbsoluteTableIdentifier,
-              newColumnIdentifier, carbonDimension.getDataType,
-              dictionaryPath)
+              newColumnIdentifier, carbonDimension.getDataType)
             allDictIdentifiers += dictionaryColumnUniqueIdentifier
             new ForwardDictionaryWrapper(dictionaryColumnUniqueIdentifier, broadcastConf)
           } catch {
@@ -608,11 +602,8 @@
     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).getAbsoluteTableIdentifier,
-            f._2, f._3.getDataType, dictionaryPath))
+            atiMap(f._1).getAbsoluteTableIdentifier, f._2, f._3.getDataType))
         } catch {
           case _: Throwable => null
         }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 8cc407b..4e560b8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -28,13 +28,12 @@
 import org.apache.hadoop.conf.Configuration
 import org.apache.log4j.Logger
 import org.apache.spark.rdd.RDD
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
 import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, UnresolvedAttribute}
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, SortOrder}
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Sort}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
 import org.apache.spark.sql.execution.LogicalRDD
 import org.apache.spark.sql.execution.command.{AtomicRunnableCommand, DataLoadTableFileMapping, UpdateTableModel}
 import org.apache.spark.sql.execution.datasources._
@@ -54,10 +53,7 @@
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.datastore.row.CarbonRow
-import org.apache.carbondata.core.dictionary.server.{DictionaryServer, NonSecureDictionaryServer}
-import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceProvider
 import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.SegmentFileStore
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
@@ -74,11 +70,9 @@
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.processing.util.{CarbonBadRecordUtil, CarbonDataProcessorUtil, CarbonLoaderUtil}
-import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
-import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
 import org.apache.carbondata.spark.load.{CsvRDDHelper, DataLoadProcessorStepOnSpark, GlobalSortHelper}
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 case class CarbonLoadDataCommand(
     databaseNameOp: Option[String],
@@ -139,7 +133,6 @@
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
-    var concurrentLoadLock: Option[ICarbonLock] = None
     carbonProperty.addProperty("zookeeper.enable.lock", "false")
     currPartitions = if (table.isHivePartitionTable) {
       CarbonFilters.getCurrentPartitions(
@@ -287,7 +280,6 @@
       }
       // First system has to partition the data first and then call the load data
       LOGGER.info(s"Initiating Direct Load for the Table : ($dbName.$tableName)")
-      concurrentLoadLock = acquireConcurrentLoadLock()
       // Clean up the old invalid segment data before creating a new entry for new load.
       SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false, currPartitions)
       // add the start entry for the new load in the table status file
@@ -300,22 +292,7 @@
       if (isOverwriteTable) {
         LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
       }
-      // if table is an aggregate table then disable single pass.
-      if (carbonLoadModel.isAggLoadRequest) {
-        carbonLoadModel.setUseOnePass(false)
-      }
 
-      // start dictionary server when use one pass load and dimension with DICTIONARY
-      // encoding is present.
-      val allDimensions =
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getAllDimensions.asScala.toList
-      val createDictionary = allDimensions.exists {
-        carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-                           !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
-      }
-      if (!createDictionary) {
-        carbonLoadModel.setUseOnePass(false)
-      }
       // Create table and metadata folders if not exist
       if (carbonLoadModel.isCarbonTransactionalTable) {
         val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
@@ -328,26 +305,14 @@
       val partitionStatus = SegmentStatus.SUCCESS
       val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
       LOGGER.info("Sort Scope : " + carbonLoadModel.getSortScope)
-      if (carbonLoadModel.getUseOnePass) {
-        loadDataUsingOnePass(
-          sparkSession,
-          carbonProperty,
-          carbonLoadModel,
-          columnar,
-          partitionStatus,
-          hadoopConf,
-          operationContext,
-          LOGGER)
-      } else {
-        loadData(
-          sparkSession,
-          carbonLoadModel,
-          columnar,
-          partitionStatus,
-          hadoopConf,
-          operationContext,
-          LOGGER)
-      }
+      loadData(
+        sparkSession,
+        carbonLoadModel,
+        columnar,
+        partitionStatus,
+        hadoopConf,
+        operationContext,
+        LOGGER)
       val loadTablePostExecutionEvent: LoadTablePostExecutionEvent =
         new LoadTablePostExecutionEvent(
           table.getCarbonTableIdentifier,
@@ -379,183 +344,10 @@
           CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uuid)
         }
         throw ex
-    } finally {
-      releaseConcurrentLoadLock(concurrentLoadLock, LOGGER)
     }
     Seq.empty
   }
 
-  private def acquireConcurrentLoadLock(): Option[ICarbonLock] = {
-    val isConcurrentLockRequired = table.getAllDimensions.asScala
-      .exists(cd => cd.hasEncoding(Encoding.DICTIONARY) &&
-                    !cd.hasEncoding(Encoding.DIRECT_DICTIONARY))
-
-    if (isConcurrentLockRequired) {
-      var concurrentLoadLock: ICarbonLock = CarbonLockFactory.getCarbonLockObj(
-        table.getTableInfo().getOrCreateAbsoluteTableIdentifier(),
-        LockUsage.CONCURRENT_LOAD_LOCK)
-      val retryCount = CarbonLockUtil
-        .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-          CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-      val maxTimeout = CarbonLockUtil
-        .getLockProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-          CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-      if (!(isConcurrentLockRequired &&
-            concurrentLoadLock.lockWithRetries(retryCount, maxTimeout))) {
-        throw new RuntimeException(table.getDatabaseName + "." + table.getTableName +
-                                   " having dictionary column. so concurrent load is not supported")
-      }
-      return Some(concurrentLoadLock)
-    }
-    return None
-  }
-
-  private def releaseConcurrentLoadLock(concurrentLoadLock: Option[ICarbonLock],
-      LOGGER: Logger): Unit = {
-    if (concurrentLoadLock.isDefined) {
-      if (concurrentLoadLock.get.unlock()) {
-        LOGGER.info("concurrent_load lock for table" + table.getTablePath +
-                    "has been released successfully")
-      } else {
-        LOGGER.error(
-          "Unable to unlock concurrent_load lock for table" + table.getTablePath);
-      }
-    }
-  }
-
-  private def loadDataUsingOnePass(
-      sparkSession: SparkSession,
-      carbonProperty: CarbonProperties,
-      carbonLoadModel: CarbonLoadModel,
-      columnar: Boolean,
-      partitionStatus: SegmentStatus,
-      hadoopConf: Configuration,
-      operationContext: OperationContext,
-      LOGGER: Logger): Seq[Row] = {
-    var rows = Seq.empty[Row]
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      .getCarbonTableIdentifier
-    val dictFolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
-    val dimensions = carbonTable.getVisibleDimensions().asScala.toArray
-    val colDictFilePath = carbonLoadModel.getColDictFilePath
-    if (!StringUtils.isEmpty(colDictFilePath)) {
-      carbonLoadModel.initPredefDictMap()
-      // generate predefined dictionary
-      GlobalDictionaryUtil.generatePredefinedColDictionary(
-        colDictFilePath,
-        carbonTableIdentifier,
-        dimensions,
-        carbonLoadModel,
-        sparkSession.sqlContext,
-        dictFolderPath)
-    }
-    if (!StringUtils.isEmpty(carbonLoadModel.getAllDictPath)) {
-      carbonLoadModel.initPredefDictMap()
-      GlobalDictionaryUtil
-        .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
-          carbonLoadModel,
-          carbonTableIdentifier,
-          dictFolderPath,
-          dimensions,
-          carbonLoadModel.getAllDictPath)
-    }
-    // dictionaryServerClient dictionary generator
-    val dictionaryServerPort = carbonProperty
-      .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-        CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
-    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-      getConf.get("spark.driver.host")
-    carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
-
-    val carbonSecureModeDictServer = CarbonProperties.getInstance.
-      getProperty(CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER,
-      CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER_DEFAULT)
-
-    val sparkConf = sparkSession.sqlContext.sparkContext.getConf
-    // For testing.
-    // sparkConf.set("spark.authenticate", "true")
-    // sparkConf.set("spark.authenticate.secret", "secret")
-
-    val server: Option[DictionaryServer] = if (sparkConf.get("spark.authenticate", "false").
-      equalsIgnoreCase("true") && carbonSecureModeDictServer.toBoolean) {
-      val dictionaryServer = SecureDictionaryServer.getInstance(sparkConf,
-        sparkDriverHost.toString, dictionaryServerPort.toInt, carbonTable)
-      carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-      carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
-      carbonLoadModel.setDictionaryServerSecretKey(dictionaryServer.getSecretKey)
-      carbonLoadModel.setDictionaryEncryptServerSecure(dictionaryServer.isEncryptSecureServer)
-      carbonLoadModel.setDictionaryServiceProvider(new SecureDictionaryServiceProvider())
-      sparkSession.sparkContext.addSparkListener(new SparkListener() {
-        override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-          dictionaryServer.shutdown()
-        }
-      })
-      Some(dictionaryServer)
-    } else {
-      val dictionaryServer = NonSecureDictionaryServer
-        .getInstance(dictionaryServerPort.toInt, carbonTable)
-      carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-      carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
-      carbonLoadModel.setDictionaryEncryptServerSecure(false)
-      carbonLoadModel
-        .setDictionaryServiceProvider(new NonSecureDictionaryServiceProvider(dictionaryServer
-          .getPort))
-      sparkSession.sparkContext.addSparkListener(new SparkListener() {
-        override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-          dictionaryServer.shutdown()
-        }
-      })
-      Some(dictionaryServer)
-    }
-    val loadDataFrame = if (updateModel.isDefined) {
-       Some(getDataFrameWithTupleID())
-    } else {
-      dataFrame
-    }
-
-    if (carbonTable.isHivePartitionTable) {
-      try {
-        rows = loadDataWithPartition(
-          sparkSession,
-          carbonLoadModel,
-          hadoopConf,
-          loadDataFrame,
-          operationContext,
-          LOGGER)
-      } finally {
-        server match {
-          case Some(dictServer) =>
-            try {
-              dictServer.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-                .getCarbonTableIdentifier.getTableId)
-            } catch {
-              case _: Exception =>
-                throw new Exception("Dataload failed due to error while writing dictionary file!")
-            }
-          case _ =>
-        }
-      }
-    } else {
-      val loadResult = CarbonDataRDDFactory.loadCarbonData(
-        sparkSession.sqlContext,
-        carbonLoadModel,
-        columnar,
-        partitionStatus,
-        server,
-        isOverwriteTable,
-        hadoopConf,
-        loadDataFrame,
-        updateModel,
-        operationContext)
-      if (loadResult != null) {
-        val info = makeAuditInfo(loadResult)
-        setAuditInfo(info)
-      }
-    }
-    rows
-  }
-
   private def makeAuditInfo(loadResult: LoadMetadataDetails): Map[String, String] = {
     if (loadResult != null) {
       Map(
@@ -587,11 +379,6 @@
       (dataFrame, dataFrame)
     }
     val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sparkSession.sqlContext,
-      carbonLoadModel,
-      hadoopConf,
-      dictionaryDataFrame)
     if (table.isHivePartitionTable) {
       rows = loadDataWithPartition(
         sparkSession,
@@ -605,7 +392,6 @@
         carbonLoadModel,
         columnar,
         partitionStatus,
-        None,
         isOverwriteTable,
         hadoopConf,
         loadDataFrame,
@@ -1114,9 +900,6 @@
     val options = new mutable.HashMap[String, String]()
     options ++= catalogTable.storage.properties
     options += (("overwrite", overWrite.toString))
-    options += (("onepass", loadModel.getUseOnePass.toString))
-    options += (("dicthost", loadModel.getDictionaryServerHost))
-    options += (("dictport", loadModel.getDictionaryServerPort.toString))
     if (partition.nonEmpty) {
       val staticPartitionStr = ObjectSerializationUtil.convertObjectToString(
         new util.HashMap[String, Boolean](
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
index dc52e5d..c12ff6c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -32,7 +32,6 @@
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.events.{AlterTableAddColumnPostEvent, AlterTableAddColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.TableInfo
-import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
 
 private[sql] case class CarbonAlterTableAddColumnCommand(
     alterTableAddColumnsModel: AlterTableAddColumnsModel)
@@ -83,10 +82,6 @@
         sparkSession.sparkContext).process
       setAuditInfo(Map(
         "newColumn" -> newCols.map(x => s"${x.getColumnName}:${x.getDataType}").mkString(",")))
-      // generate dictionary files for the newly added columns
-      new AlterTableAddColumnRDD(sparkSession,
-        newCols,
-        carbonTable.getAbsoluteTableIdentifier).collect()
       timeStamp = System.currentTimeMillis
       val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
       schemaEvolutionEntry.setTimeStamp(timeStamp)
@@ -126,9 +121,6 @@
       case e: Exception =>
         if (newCols.nonEmpty) {
           LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
-          new AlterTableDropColumnRDD(sparkSession,
-            newCols,
-            carbonTable.getAbsoluteTableIdentifier).collect()
           AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
         }
         throwMetadataException(dbName, tableName,
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 3a6f1f5..bdc0228 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -34,7 +34,6 @@
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.events.{AlterTableDropColumnPostEvent, AlterTableDropColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
-import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
 
 private[sql] case class CarbonAlterTableDropColumnCommand(
     alterTableDropColumnModel: AlterTableDropColumnModel)
@@ -169,10 +168,6 @@
         .alterDropColumns(tableIdentifier, schemaParts, columns)
       sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
       // TODO: 1. add check for deletion of index tables
-      // delete dictionary files for dictionary column and clear dictionary cache from memory
-      new AlterTableDropColumnRDD(sparkSession,
-        dictionaryColumns,
-        carbonTable.getAbsoluteTableIdentifier).collect()
 
       // event will be fired before dropping the columns
       val alterTableDropColumnPostEvent: AlterTableDropColumnPostEvent =
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
index 8810425..0c98052 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
@@ -145,8 +145,6 @@
       ("", "", ""),
       ("## Encoding Information", "", ""))
     results ++= getLocalDictDesc(carbonTable, tblProps.toMap)
-    results ++= Seq(("Global Dictionary",
-      tblProps.getOrElse(CarbonCommonConstants.DICTIONARY_INCLUDE, ""), ""))
     if (tblProps.contains(CarbonCommonConstants.LONG_STRING_COLUMNS)) {
       results ++= Seq((CarbonCommonConstants.LONG_STRING_COLUMNS.toUpperCase,
         tblProps.getOrElse(CarbonCommonConstants.LONG_STRING_COLUMNS, ""), ""))
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index ba84262..54a5757 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -28,7 +28,6 @@
 import org.apache.spark.sql.hive.CarbonFileMetastore
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -195,7 +194,6 @@
   override def processData(sparkSession: SparkSession): Seq[Row] = {
     // clear driver side index and dictionary cache
     if (carbonTable != null && !(carbonTable.isChildTableForMV && !dropChildTable)) {
-      ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
       // delete the table folder
       val tablePath = carbonTable.getTablePath
       // delete table data only if it is not external table
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
index 1fd155f..bde74fd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
@@ -121,9 +121,6 @@
       optionsFinal,
       model,
       conf)
-    model.setUseOnePass(options.getOrElse("onepass", "false").toBoolean)
-    model.setDictionaryServerHost(options.getOrElse("dicthost", null))
-    model.setDictionaryServerPort(options.getOrElse("dictport", "-1").toInt)
     CarbonTableOutputFormat.setOverwrite(conf, options("overwrite").toBoolean)
     model.setLoadWithoutConverterStep(true)
     val staticPartition = options.getOrElse("staticpartition", null)
@@ -589,10 +586,7 @@
       }
       if (staticPartition != null && staticPartition.get(col.getColumnName.toLowerCase)) {
         val converetedVal =
-          CarbonScalaUtil.convertStaticPartitions(
-            partitionData(index),
-            col,
-            model.getCarbonDataLoadSchema.getCarbonTable)
+          CarbonScalaUtil.convertStaticPartitions(partitionData(index), col)
         if (col.hasEncoding(Encoding.DICTIONARY)) {
           converetedVal.toInt.asInstanceOf[AnyRef]
         } else {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index b667a57..2e1f91f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -595,14 +595,15 @@
 
   private def hasMoreDictionaryColumnsOnProjection(projectColumns: AttributeSet,
       relation: CarbonDatasourceHadoopRelation): Boolean = {
-    val map = relation.carbonRelation.metaData.dictionaryMap
-    var count = 0
-    projectColumns.foreach{c =>
-      if (map.get(c.name).getOrElse(false)) {
-        count += 1
-      }
-    }
-    count > CarbonCommonConstants.CARBON_ALLOW_DIRECT_FILL_DICT_COLS_LIMIT
+//    val map = relation.carbonRelation.metaData.dictionaryMap
+//    var count = 0
+//    projectColumns.foreach{c =>
+//      if (map.get(c.name).getOrElse(false)) {
+//        count += 1
+//      }
+//    }
+//    count > CarbonCommonConstants.CARBON_ALLOW_DIRECT_FILL_DICT_COLS_LIMIT
+    false
   }
 
   private def getPartitioning(carbonTable: CarbonTable,
@@ -610,8 +611,6 @@
     val info: BucketingInfo = carbonTable.getBucketingInfo()
     if (info != null) {
       val cols = info.getListOfColumns.asScala
-      val sortColumn = carbonTable.
-              getVisibleDimensions().get(0).getColName
       val numBuckets = info.getNumOfRanges
       val bucketColumns = cols.flatMap { n =>
         val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 57755da..b2ba7f4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -37,7 +37,6 @@
 import org.apache.spark.util.{CarbonReflectionUtils, SparkUtil}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesAndSchemaHolder
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -455,11 +454,6 @@
   def dropTable(absoluteTableIdentifier: AbsoluteTableIdentifier)(sparkSession: SparkSession) {
     val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
-    val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName, tableName)
-    if (null != carbonTable) {
-      // clear driver B-tree and dictionary cache
-      ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
-    }
     CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
     // discard cached table info in cachedDataSourceTables
     val tableIdentifier = TableIdentifier(tableName, Option(dbName))
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index 9067d28..f2133cc 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -18,18 +18,16 @@
 
 import scala.collection.JavaConverters._
 
-import org.apache.hadoop.fs.Path
-import org.apache.spark.sql.{CarbonEnv, CarbonSession, SparkSession}
+import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.catalyst.TableIdentifier
 
-import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesAndSchemaHolder
-import org.apache.carbondata.core.metadata.{schema, AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.format
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -75,11 +73,6 @@
     (sparkSession: SparkSession): Unit = {
     val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
-    val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName, tableName)
-    if (null != carbonTable) {
-      // clear driver B-tree and dictionary cache
-      ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
-    }
     CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
     // discard cached table info in cachedDataSourceTables
     val tableIdentifier = TableIdentifier(tableName, Option(dbName))
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 7797b42..69145d8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -546,7 +546,7 @@
     (TBLPROPERTIES ~> "(" ~> repsep(loadOptions, ",") <~ ")").? <~ opt(";") ^^ {
       case dbName ~ table ~ fields ~ tblProp =>
         fields.foreach{ f =>
-          if (isComplexDimDictionaryExclude(f.dataType.get)) {
+          if (isComplexType(f.dataType.get)) {
             throw new MalformedCarbonCommandException(
               s"Add column is unsupported for complex datatype column: ${f.column}")
           }
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
index 60e5cad..be3ac52 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
@@ -30,6 +30,7 @@
 import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
 import org.apache.spark.sql.types.StructField
 
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -134,7 +135,12 @@
         "Table properties are not supported for external table", tablePropertyList)
     }
 
-    // validate tblProperties
+    // validate tblProperties, global dictionary is deprecated
+    if (tableProperties.contains(CarbonCommonConstants.DICTIONARY_INCLUDE) ||
+        tableProperties.contains(CarbonCommonConstants.DICTIONARY_EXCLUDE)) {
+      throw new DeprecatedFeatureException("global dictionary")
+    }
+
     val bucketFields = parser.getBucketFields(tableProperties, fields, options)
     var isTransactionalTable: Boolean = true
 
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 61eb2dd..b7b1be4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -891,9 +891,7 @@
   def ValidateSetTablePropertiesForLocalDict(tblPropertiesMap: mutable.Map[String, String],
       carbonTable: CarbonTable,
       property: (String, String)): Unit = {
-    var primitiveComplexChildColumns = new mutable.HashSet[String]
     var localDictColumns: Seq[String] = Seq[String]()
-    var dictIncludeColumns: Seq[String] = Seq[String]()
 
     val allColumns = carbonTable.getTableInfo.getFactTable.getListOfColumns.asScala
     localDictColumns = property._2.toString.toLowerCase.split(",").map(_.trim)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
index 0333cab..ab7fb15 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
@@ -125,20 +125,10 @@
     val parentDictExclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
       .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, "").split(",")
 
-    val parentGlobalDictInclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
-      .getOrElse(CarbonCommonConstants.DICTIONARY_INCLUDE, "").split(",")
-
-    val parentGlobalDictExclude = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
-      .getOrElse(CarbonCommonConstants.DICTIONARY_EXCLUDE, "").split(",")
-
     val newLocalDictInclude = getDataMapColumns(parentDictInclude, fields, fieldRelationMap)
 
     val newLocalDictExclude = getDataMapColumns(parentDictExclude, fields, fieldRelationMap)
 
-    val newGlobalDictInclude = getDataMapColumns(parentGlobalDictInclude, fields, fieldRelationMap)
-
-    val newGlobalDictExclude = getDataMapColumns(parentGlobalDictExclude, fields, fieldRelationMap)
-
     if (newLocalDictInclude.nonEmpty) {
       tableProperties
         .put(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE, newLocalDictInclude.mkString(","))
@@ -148,15 +138,6 @@
         .put(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE, newLocalDictExclude.mkString(","))
     }
 
-    if (newGlobalDictInclude.nonEmpty) {
-      tableProperties
-        .put(CarbonCommonConstants.DICTIONARY_INCLUDE, newGlobalDictInclude.mkString(","))
-    }
-    if (newGlobalDictExclude.nonEmpty) {
-      tableProperties
-        .put(CarbonCommonConstants.DICTIONARY_EXCLUDE, newGlobalDictExclude.mkString(","))
-    }
-
     val parentInvertedIndex = parentTable.getTableInfo.getFactTable.getTableProperties.asScala
       .getOrElse(CarbonCommonConstants.INVERTED_INDEX, "").split(",")
 
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSQLConf.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSQLConf.scala
index 413973e..8eb05fc 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSQLConf.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSQLConf.scala
@@ -72,11 +72,6 @@
         .stringConf
         .createWithDefault(carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
           CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    val SINGLE_PASS =
-      SQLConfigBuilder(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS)
-        .doc("Property to enable/disable single_pass.")
-        .booleanConf
-        .createWithDefault(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
     val BAD_RECORD_PATH =
       SQLConfigBuilder(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH)
         .doc("Property to configure the bad record location.")
@@ -122,8 +117,6 @@
     sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
       carbonProperties.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
         CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-      CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT.toBoolean)
     sparkSession.conf.set(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
       carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
         CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
index ad379fb..c1a5862 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapFunctionSuite.scala
@@ -49,7 +49,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(id INT, name STRING, city STRING, age INT,
          | s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING, s7 STRING, s8 STRING)
-         | STORED BY 'carbondata' TBLPROPERTIES('table_blocksize'='128', 'dictionary_include'='id', 'sort_columns'='id')
+         | STORED BY 'carbondata' TBLPROPERTIES('table_blocksize'='128', 'sort_columns'='id')
          |  """.stripMargin)
     sql(
       s"""
@@ -95,7 +95,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(id INT, name STRING, city STRING, age INT,
          | s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING, s7 STRING, s8 STRING)
-         | STORED BY 'carbondata' TBLPROPERTIES('table_blocksize'='128', 'dictionary_include'='id', 'sort_columns'='name')
+         | STORED BY 'carbondata' TBLPROPERTIES('table_blocksize'='128', 'sort_columns'='name')
          |  """.stripMargin)
     sql(
       s"""
@@ -232,7 +232,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(empno string, doj date, salary float)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('SORT_COLUMNS'='empno', 'dictionary_include'='salary')
+         | TBLPROPERTIES('SORT_COLUMNS'='empno')
        """.stripMargin)
     sql(
       s"""
@@ -310,7 +310,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(empno string, doj date, salary float)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('SORT_COLUMNS'='empno', 'dictionary_include'='doj', 'sort_columns'='doj')
+         | TBLPROPERTIES('SORT_COLUMNS'='empno', 'sort_columns'='doj')
        """.stripMargin)
     sql(
       s"""
@@ -330,8 +330,6 @@
       sql(s"SELECT * FROM $normalTable WHERE doj='2016-03-15'"))
   }
 
-  // since date cannot be dictionary_exclude, we skip the test case
-
   // timestamp is naturally not dictionary
   test("test bloom datamap: index column is timestamp") {
     val timeStampData = s"$resourcesPath/timeStampFormatData1.csv"
@@ -472,7 +470,7 @@
          | CREATE TABLE $bloomDMSampleTable(id INT, name STRING, city STRING, age INT,
          | s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING, s7 STRING, s8 STRING)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('table_blocksize'='128', 'dictionary_include'='id, name, s1', 'sort_columns'='id')
+         | TBLPROPERTIES('table_blocksize'='128', 'sort_columns'='id')
          |  """.stripMargin)
 
     sql(
@@ -513,7 +511,7 @@
          | CREATE TABLE $bloomDMSampleTable(id INT, name STRING, city STRING, age INT,
          | s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING, s7 STRING, s8 STRING)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('table_blocksize'='128', 'dictionary_include'='id, name, s1', 'sort_columns'='name')
+         | TBLPROPERTIES('table_blocksize'='128', 'sort_columns'='name')
          |  """.stripMargin)
 
     sql(
@@ -554,7 +552,7 @@
          | CREATE TABLE $bloomDMSampleTable(id INT, name STRING, city STRING, age INT,
          | s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING, s7 STRING, s8 STRING)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('table_blocksize'='128', 'dictionary_include'='name, s1', 'sort_columns'='id')
+         | TBLPROPERTIES('table_blocksize'='128', 'sort_columns'='id')
          |  """.stripMargin)
 
     sql(
@@ -633,7 +631,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(empno string, doj date, salary float)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('SORT_COLUMNS'='empno', 'DICTIONARY_INCLUDE'='salary')
+         | TBLPROPERTIES('SORT_COLUMNS'='empno')
        """.stripMargin)
     sql(
       s"""
@@ -709,7 +707,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(empno string, doj date, salary float)
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('SORT_COLUMNS'='empno,doj', 'DICTIONARY_INCLUDE'='doj,empno')
+         | TBLPROPERTIES('SORT_COLUMNS'='empno,doj')
        """.stripMargin)
     sql(
       s"""
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
index bedcc4a..5c7cc0b 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/datamap/bloom/BloomCoarseGrainDataMapSuite.scala
@@ -619,8 +619,7 @@
       s"""
          | CREATE TABLE $bloomDMSampleTable(id INT, name STRING, city STRING, age INT,
          | s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING, s7 STRING, s8 STRING)
-         | STORED BY 'carbondata' TBLPROPERTIES('table_blocksize'='128',
-         | 'DICTIONARY_INCLUDE'='s1,s2', 'CACHE_LEVEL'='BLOCKLET')
+         | STORED BY 'carbondata' TBLPROPERTIES('table_blocksize'='128','CACHE_LEVEL'='BLOCKLET')
          |  """.stripMargin)
 
     // load data into table (segment0)
@@ -643,9 +642,7 @@
       s"""
          | ALTER TABLE $bloomDMSampleTable
          | ADD COLUMNS(num1 INT, dictString STRING, noDictString STRING)
-         | TBLPROPERTIES('DEFAULT.VALUE.num1'='999', 'DEFAULT.VALUE.dictString'='old',
-         | 'DICTIONARY_INCLUDE'='dictString'
-         | )
+         | TBLPROPERTIES('DEFAULT.VALUE.num1'='999', 'DEFAULT.VALUE.dictString'='old')
          """.stripMargin)
 
     // load data into table (segment1)
@@ -902,7 +899,6 @@
              | TBLPROPERTIES(
              | 'SORT_COLUMNS'='market_code, device_code, country_code, category_id, date,product_id',
              | 'NO_INVERTED_INDEX'='est_free_app_download, est_paid_app_download,est_revenue',
-             | 'DICTIONARY_INCLUDE' = 'market_code, device_code, country_code,category_id, product_id',
              | 'SORT_SCOPE'='GLOBAL_SORT',
              | 'CACHE_LEVEL'='BLOCKLET',  'TABLE_BLOCKSIZE'='256',
              | 'GLOBAL_SORT_PARTITIONS'='2'
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesBaseTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesBaseTest.scala
index 82894d4..bde6e4a 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesBaseTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesBaseTest.scala
@@ -82,7 +82,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
     checkExistence(sql("describe formatted carbon_table"), true, "boolean")
   }
@@ -157,7 +157,7 @@
 
   test("test boolean as dictionary include column and codegen=false"){
     sql("drop table if exists carbon_table")
-    sql("create table carbon_table(a1 boolean,a2 string,a3 int) stored by 'carbondata' tblproperties('dictionary_include'='a1')")
+    sql("create table carbon_table(a1 boolean,a2 string,a3 int) stored by 'carbondata' ")
     sql("insert into carbon_table select false,'a',1")
     sql("set spark.sql.codegen.wholestage=false")
     checkAnswer(sql("select a1 from carbon_table"), Seq(Row(false)))
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
index 466475b..61acd81 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesFilterTest.scala
@@ -58,7 +58,6 @@
          | complexData ARRAY<STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
        """.stripMargin)
     sql(
       s"""
@@ -366,7 +365,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+
        """.stripMargin)
     sql(
       s"""
@@ -408,7 +407,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+
        """.stripMargin)
     sql(
       s"""
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
index f63dc59..66b7158 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
@@ -146,7 +146,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -167,7 +167,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -263,7 +263,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -284,7 +284,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val rootPath = new File(this.getClass.getResource("/").getPath
@@ -334,7 +334,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -354,7 +354,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val rootPath = new File(this.getClass.getResource("/").getPath
@@ -390,7 +390,7 @@
            | complexData ARRAY<STRING>
            | )
            | STORED BY 'carbondata'
-           | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+           | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
       sql(
@@ -411,7 +411,7 @@
            | booleanField2 BOOLEAN
            | )
            | STORED BY 'carbondata'
-           | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+           | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
       val rootPath = new File(this.getClass.getResource("/").getPath
@@ -448,7 +448,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -541,7 +541,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val rootPath = new File(this.getClass.getResource("/").getPath
@@ -699,7 +699,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -720,7 +720,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -816,7 +816,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -909,7 +909,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val rootPath = new File(this.getClass.getResource("/").getPath
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
index 93255fa..3e4107e 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesLoadTest.scala
@@ -73,7 +73,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
   }
 
@@ -171,7 +171,7 @@
          | complexData ARRAY<STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBooleanTwoBooleanColumns.csv"
@@ -209,7 +209,7 @@
          | complexData ARRAY<STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBooleanWithFileHeader.csv"
@@ -227,7 +227,7 @@
     )
   }
 
-  test("Loading table: create with DICTIONARY_EXCLUDE, TABLE_BLOCKSIZE, NO_INVERTED_INDEX, SORT_SCOPE") {
+  test("Loading table: create with TABLE_BLOCKSIZE, NO_INVERTED_INDEX, SORT_SCOPE") {
     sql("drop table if exists boolean_table")
     sql(
       s"""
@@ -247,7 +247,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('DICTIONARY_EXCLUDE'='charField','TABLE_BLOCKSIZE'='512','NO_INVERTED_INDEX'='charField', 'SORT_SCOPE'='GLOBAL_SORT')
+         | TBLPROPERTIES('TABLE_BLOCKSIZE'='512','NO_INVERTED_INDEX'='charField', 'SORT_SCOPE'='GLOBAL_SORT')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBoolean.csv"
@@ -318,7 +318,7 @@
     }
   }
 
-  test("Loading table: load with DELIMITER, QUOTECHAR, COMMENTCHAR, MULTILINE, ESCAPECHAR, COMPLEX_DELIMITER_LEVEL_1, SINGLE_PASS") {
+  test("Loading table: load with DELIMITER, QUOTECHAR, COMMENTCHAR, MULTILINE, ESCAPECHAR, COMPLEX_DELIMITER_LEVEL_1") {
     sql("drop table if exists boolean_table")
     sql(
       s"""
@@ -337,7 +337,7 @@
          | complexData ARRAY<STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('DICTIONARY_EXCLUDE'='charField','TABLE_BLOCKSIZE'='512','NO_INVERTED_INDEX'='charField', 'SORT_SCOPE'='GLOBAL_SORT')
+         | TBLPROPERTIES('TABLE_BLOCKSIZE'='512','NO_INVERTED_INDEX'='charField', 'SORT_SCOPE'='GLOBAL_SORT')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBooleanWithFileHeader.csv"
@@ -345,7 +345,7 @@
       s"""
          | LOAD DATA LOCAL INPATH '${storeLocation}'
          | INTO TABLE boolean_table
-         | options('DELIMITER'=',','QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='#','COMPLEX_DELIMITER_LEVEL_2'=':','SINGLE_PASS'='TRUE')
+         | options('DELIMITER'=',','QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='#','COMPLEX_DELIMITER_LEVEL_2'=':')
            """.stripMargin)
 
     checkAnswer(
@@ -505,7 +505,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     sql(
@@ -655,7 +655,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBooleanTwoBooleanColumns.csv"
@@ -715,7 +715,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBoolean.csv"
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesParameterTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesParameterTest.scala
index cf6e21e..9286636 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesParameterTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesParameterTest.scala
@@ -138,7 +138,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBoolean.csv"
@@ -183,7 +183,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='')
        """.stripMargin)
 
     val storeLocation = s"$rootPath/integration/spark2/src/test/resources/bool/supportBooleanTwoBooleanColumns.csv"
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesSortTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesSortTest.scala
index 71c6c18..3e41278 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesSortTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesSortTest.scala
@@ -54,7 +54,7 @@
          | complexData ARRAY<STRING>
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='booleanField','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='booleanField')
        """.stripMargin)
 
     sql(
@@ -107,7 +107,7 @@
          | booleanField2 BOOLEAN
          | )
          | STORED BY 'carbondata'
-         | TBLPROPERTIES('sort_columns'='shortField,booleanField,booleanField2','DICTIONARY_INCLUDE'='dateField, charField')
+         | TBLPROPERTIES('sort_columns'='shortField,booleanField,booleanField2')
        """.stripMargin)
 
     sql(
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
index 1bbb1b3..e69de29 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.util
-
-import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
-import org.apache.carbondata.processing.util.TableOptionConstant
-
-/**
-  * Test Case for org.apache.carbondata.integration.spark.util.GlobalDictionaryUtil
-  */
-class AllDictionaryTestCase extends Spark2QueryTest with BeforeAndAfterAll {
-  var pwd: String = _
-  var sampleRelation: CarbonRelation = _
-  var complexRelation: CarbonRelation = _
-  var sampleAllDictionaryFile: String = _
-  var complexAllDictionaryFile: String = _
-
-  def buildCarbonLoadModel(relation: CarbonRelation,
-    filePath: String,
-    header: String,
-    allDictFilePath: String): CarbonLoadModel = {
-    val carbonLoadModel = new CarbonLoadModel
-    carbonLoadModel.setTableName(relation.carbonTable.getDatabaseName)
-    carbonLoadModel.setDatabaseName(relation.carbonTable.getTableName)
-    carbonLoadModel.setTablePath(relation.metaData.carbonTable.getTablePath)
-    val table = relation.carbonTable
-    val carbonSchema = new CarbonDataLoadSchema(table)
-    carbonLoadModel.setDatabaseName(table.getDatabaseName)
-    carbonLoadModel.setTableName(table.getTableName)
-    carbonLoadModel.setCarbonDataLoadSchema(carbonSchema)
-    carbonLoadModel.setFactFilePath(filePath)
-    carbonLoadModel.setCsvHeader(header)
-    carbonLoadModel.setCsvDelimiter(",")
-    carbonLoadModel.setComplexDelimiter("$")
-    carbonLoadModel.setComplexDelimiter(":")
-    carbonLoadModel.setAllDictPath(allDictFilePath)
-    carbonLoadModel.setSerializationNullFormat(
-          TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + ",\\N")
-    carbonLoadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-    carbonLoadModel.setCsvHeaderColumns(
-      LoadOption.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
-    // Create table and metadata folders if not exist
-    val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
-    if (!FileFactory.isFileExist(metadataDirectoryPath)) {
-      FileFactory.mkdirs(metadataDirectoryPath)
-    }
-    import scala.collection.JavaConverters._
-    val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
-      .getOrElse(CarbonCommonConstants.COMPRESSOR,
-        CompressorFactory.getInstance().getCompressor.getName)
-    carbonLoadModel.setColumnCompressor(columnCompressor)
-    carbonLoadModel
-  }
-
-  override def beforeAll {
-    sql("drop table if exists sample")
-    sql("drop table if exists complextypes")
-    sql("drop table if exists tabletest")
-    buildTestData
-    // second time comment this line
-    buildTable
-    buildRelation
-  }
-
-  def buildTestData() = {
-    sampleAllDictionaryFile = s"${resourcesPath}/alldictionary/sample/20160423/1400_1405/*.dictionary"
-    complexAllDictionaryFile = s"${resourcesPath}/alldictionary/complex/20160423/1400_1405/*.dictionary"
-  }
-
-  def buildTable() = {
-    try {
-      sql(
-        "CREATE TABLE IF NOT EXISTS sample (id STRING, name STRING, city STRING, " +
-          "age INT) STORED BY 'org.apache.carbondata.format' " +
-          "TBLPROPERTIES('dictionary_include'='city')"
-      )
-    } catch {
-      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
-    }
-    try {
-      sql(
-        "create table complextypes (deviceInformationId string, channelsId string, " +
-          "ROMSize string, purchasedate string, mobile struct<imei: string, imsi: string>, MAC " +
-          "array<string>, locationinfo array<struct<ActiveAreaId: INT, ActiveCountry: string, " +
-          "ActiveProvince: string, Activecity: string, ActiveDistrict: string, ActiveStreet: " +
-          "string>>, proddate struct<productionDate: string,activeDeactivedate: array<string>>, " +
-          "gamePointId INT,contractNumber INT) STORED BY 'org.apache.carbondata.format'" +
-          "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ROMSize', 'dictionary_include'='channelsId')"
-      )
-    } catch {
-      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
-    }
-  }
-
-  def buildRelation() = {
-    val warehouse = s"$resourcesPath/target/warehouse"
-    val storeLocation = s"$resourcesPath/target/store"
-    val metaStoreDB = s"$resourcesPath/target"
-    CarbonProperties.getInstance()
-      .addProperty("carbon.custom.distribution", "true")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,"FORCE")
-    import org.apache.spark.sql.CarbonSession._
-
-    val spark = SparkSession
-      .builder()
-      .master("local")
-      .appName("CarbonSessionExample")
-      .config("spark.sql.warehouse.dir", warehouse)
-      .config("spark.network.timeout", "600s")
-      .config("spark.executor.heartbeatInterval", "600s")
-      .config("carbon.enable.vector.reader","false")
-      .getOrCreateCarbonSession(storeLocation, metaStoreDB)
-    val catalog = CarbonEnv.getInstance(spark).carbonMetaStore
-    sampleRelation = catalog.lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "sample")(spark).asInstanceOf[CarbonRelation]
-    complexRelation = catalog.lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "complextypes")(spark).asInstanceOf[CarbonRelation]
-  }
-
-  test("Support generate global dictionary from all dictionary files") {
-    val header = "id,name,city,age"
-    val carbonLoadModel = buildCarbonLoadModel(sampleRelation, null, header, sampleAllDictionaryFile)
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sqlContext,
-      carbonLoadModel,
-      FileFactory.getConfiguration)
-
-    DictionaryTestCaseUtil.
-      checkDictionary(sampleRelation, "city", "shenzhen")
-  }
-
-  test("Support generate global dictionary from all dictionary files for complex type") {
-    val header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC,locationinfo,proddate,gamePointId,contractNumber"
-    val carbonLoadModel = buildCarbonLoadModel(complexRelation, null, header, complexAllDictionaryFile)
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sqlContext,
-      carbonLoadModel,
-      FileFactory.getConfiguration)
-
-    DictionaryTestCaseUtil.
-      checkDictionary(complexRelation, "channelsId", "1650")
-  }
-
-  test("test create table thorugh 'using carbondata' and load data") {
-    sql(
-      "CREATE TABLE tabletest (empno INT, workgroupcategory STRING, deptno INT, projectcode INT, " +
-      "attendance INT) USING carbondata")
-    sql(
-      s"""LOAD DATA LOCAL INPATH '$resourcesPath/data.csv' INTO TABLE tabletest OPTIONS
-         |('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'FILEHEADER'='')""".stripMargin)
-    checkAnswer(sql("select count(*) from tabletest"), Row(10))
-  }
-
-  override def afterAll {
-    sql("drop table sample")
-    sql("drop table complextypes")
-    sql("drop table tabletest")
-  }
-}
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryLRUCacheTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryLRUCacheTestCase.scala
deleted file mode 100644
index 8cbae2e..0000000
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryLRUCacheTestCase.scala
+++ /dev/null
@@ -1,375 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.util
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.Spark2TestQueryExecutor
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-
-/**
-  * Test Case for Dictionary LRU Cache.
-  */
-class DictionaryLRUCacheTestCase extends Spark2QueryTest with BeforeAndAfterAll {
-  var spark : SparkSession = null
-  var path : String = null
-
-  def checkDictionaryAccessCount(databaseName: String, tableName: String): Unit = {
-    val carbonTable = CarbonEnv.getInstance(Spark2TestQueryExecutor.spark).carbonMetaStore
-      .lookupRelation(Option(databaseName), tableName)(Spark2TestQueryExecutor.spark)
-      .asInstanceOf[CarbonRelation].carbonTable
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-
-    val dimensions = carbonTable.getAllDimensions.asScala.toList
-    dimensions.foreach { dim =>
-      val columnIdentifier = dim.getColumnIdentifier
-      // Check the dictionary cache access.
-      val identifier: DictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
-        absoluteTableIdentifier,
-        columnIdentifier,
-        columnIdentifier.getDataType)
-
-      val isDictExists: Boolean = CarbonUtil.isFileExistsForGivenColumn(identifier)
-      var dictionary: Dictionary = null
-      if (isDictExists) {
-        val dictCacheReverse: Cache[DictionaryColumnUniqueIdentifier, Dictionary]
-        = CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY)
-        dictionary = dictCacheReverse.get(identifier)
-        assert(dictionary.getAccessCount == 1)
-        CarbonUtil.clearDictionaryCache(dictionary)
-
-        val dictCacheForward: Cache[DictionaryColumnUniqueIdentifier, Dictionary]
-        = CacheProvider.getInstance().createCache(CacheType.FORWARD_DICTIONARY)
-        dictionary = dictCacheForward.get(identifier)
-        assert(dictionary.getAccessCount == 1)
-        CarbonUtil.clearDictionaryCache(dictionary)
-      }
-    }
-  }
-
-
-  override def beforeAll {
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "1")
-      .addProperty(CarbonCommonConstants.CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE, "1")
-
-    path = s"$resourcesPath/restructure/data_2000.csv"
-
-    sql("use default")
-    sql("drop table if exists carbon_new1")
-    sql("drop table if exists carbon_new2")
-    sql("drop table if exists carbon_new3")
-    sql("drop table if exists carbon_new4")
-    sql("drop table if exists carbon_new5")
-    sql("drop table if exists carbon_new6")
-    sql("drop table if exists carbon_new7")
-    sql("drop table if exists carbon_new8")
-    sql("drop table if exists carbon_new9")
-    sql("drop table if exists carbon_new10")
-  }
-
-  test("test for dictionary LRU Cache for Load Single Pass") {
-
-    sql(
-        "CREATE TABLE carbon_new1 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new1 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new1 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        "CREATE TABLE carbon_new2 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new2 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new2 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='TRUE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    checkDictionaryAccessCount("default", "carbon_new2")
-  }
-
-  test("test for dictionary LRU Cache for Load Non Single Pass") {
-
-    sql(
-        "CREATE TABLE carbon_new3 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new3 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new3 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        "CREATE TABLE carbon_new4 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new4 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new4 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    checkDictionaryAccessCount("default", "carbon_new4")
-  }
-
-  test("test for dictionary LRU Cache for Select On Table") {
-
-    sql(
-        "CREATE TABLE carbon_new5 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new5 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new5 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql("select * from carbon_new5")
-
-    checkDictionaryAccessCount("default", "carbon_new5")
-
-
-    sql(
-        "CREATE TABLE carbon_new6 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new6 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new6 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql("select * from carbon_new6")
-
-    checkDictionaryAccessCount("default", "carbon_new6")
-  }
-
-  test("test for dictionary LRU Cache for Select With Filter On Table") {
-
-    sql(
-        "CREATE TABLE carbon_new7 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new7 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new7 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql("select * from carbon_new7 where CUST_ID > 10")
-
-    checkDictionaryAccessCount("default", "carbon_new7")
-
-
-    sql(
-        "CREATE TABLE carbon_new8 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new8 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new8 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql("select * from carbon_new8 where CUST_ID > 100")
-
-    checkDictionaryAccessCount("default", "carbon_new8")
-  }
-
-  test("test for dictionary LRU Cache for Insert Into") {
-
-    sql(
-        "CREATE TABLE carbon_new9 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME, ACTIVE_EMUI_VERSION,BIGINT_COLUMN1,Double_COLUMN1, " +
-        "Double_COLUMN2')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new9 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new9 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    sql("select * from carbon_new9 where CUST_ID > 10")
-
-    checkDictionaryAccessCount("default", "carbon_new9")
-
-
-    sql(
-        "CREATE TABLE carbon_new10 (CUST_ID INT,CUST_NAME STRING,ACTIVE_EMUI_VERSION STRING, DOB " +
-        "TIMESTAMP, DOJ TIMESTAMP, BIGINT_COLUMN1 BIGINT,BIGINT_COLUMN2 BIGINT,DECIMAL_COLUMN1 " +
-        "decimal(30,10), DECIMAL_COLUMN2 DECIMAL(36,10),Double_COLUMN1 double, Double_COLUMN2 " +
-        "double,INTEGER_COLUMN1 INT) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES " +
-        "('dictionary_include'='CUST_NAME')")
-
-    sql("insert into carbon_new10 select * from carbon_new9")
-
-    checkDictionaryAccessCount("default", "carbon_new10")
-
-    sql(
-        s"LOAD DATA INPATH '$path' INTO TABLE carbon_new10 OPTIONS" +
-        "('DELIMITER'=',' , 'QUOTECHAR'='\"','BAD_RECORDS_ACTION'='FORCE','SINGLE_PASS'='FALSE'," +
-        "'FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1," +
-        "BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2, " +
-        "INTEGER_COLUMN1')")
-
-    checkDictionaryAccessCount("default", "carbon_new10")
-
-  }
-
-
-
-  override def afterAll {
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE,
-        CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
-      .addProperty(CarbonCommonConstants.CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE,
-        CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT)
-
-    sql("drop table if exists carbon_new1")
-    sql("drop table if exists carbon_new2")
-    sql("drop table if exists carbon_new3")
-    sql("drop table if exists carbon_new4")
-    sql("drop table if exists carbon_new5")
-    sql("drop table if exists carbon_new6")
-    sql("drop table if exists carbon_new7")
-    sql("drop table if exists carbon_new8")
-    sql("drop table if exists carbon_new9")
-    sql("drop table if exists carbon_new10")
-  }
-}
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
deleted file mode 100644
index 5109ad2..0000000
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util
-
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.test.TestQueryExecutor
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.processing.util.CarbonLoaderUtil
-
-/**
- * Utility for global dictionary test cases
- */
-object DictionaryTestCaseUtil {
-
-  /**
-   *  check whether the dictionary of specified column generated
-   * @param relation  carbon table relation
-   * @param columnName  name of specified column
-   * @param value  a value of column
-   */
-  def checkDictionary(relation: CarbonRelation, columnName: String, value: String) {
-    val table = relation.carbonTable
-    val dimension = table.getDimensionByName(columnName)
-    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)
-    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier)
-    assert(dict.getSurrogateKey(value) != CarbonCommonConstants.INVALID_SURROGATE_KEY)
-  }
-}
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 41aa8ed..e69de29 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -1,304 +0,0 @@
-/*
-  * Licensed to the Apache Software Foundation (ASF) under one
-  * or more contributor license agreements.  See the NOTICE file
-  * distributed with this work for additional information
-  * regarding copyright ownership.  The ASF licenses this file
-  * to you under the Apache License, Version 2.0 (the
-  * "License"); you may not use this file except in compliance
-  * with the License.  You may obtain a copy of the License at
-  *
-  *    http://www.apache.org/licenses/LICENSE-2.0
-  *
-  * Unless required by applicable law or agreed to in writing,
-  * software distributed under the License is distributed on an
-  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-  * KIND, either express or implied.  See the License for the
-  * specific language governing permissions and limitations
-  * under the License.
-  */
-package org.apache.carbondata.spark.util
-
-import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.processing.exception.DataLoadingException
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
-import org.apache.carbondata.processing.util.TableOptionConstant
-
-/**
- * test case for external column dictionary generation
- * also support complicated type
- */
-class ExternalColumnDictionaryTestCase extends Spark2QueryTest with BeforeAndAfterAll {
-  var extComplexRelation: CarbonRelation = _
-  var verticalDelimiteRelation: CarbonRelation = _
-  var loadSqlRelation: CarbonRelation = _
-  var filePath: String = _
-  var pwd: String = _
-  var complexFilePath1: String = _
-  var complexFilePath2: String = _
-  var extColDictFilePath1: String = _
-  var extColDictFilePath2: String = _
-  var extColDictFilePath3: String = _
-  var header: String = _
-  var header2: String = _
-
-  def buildTestData() = {
-
-    filePath = s"${ resourcesPath }/sample.csv"
-    complexFilePath1 = s"${ resourcesPath }/complexdata2.csv"
-    complexFilePath2 = s"${ resourcesPath }/verticalDelimitedData.csv"
-    extColDictFilePath1 = s"deviceInformationId:${ resourcesPath }/deviceInformationId.csv," +
-                          s"mobile.imei:${ resourcesPath }/mobileimei.csv," +
-                          s"mac:${ resourcesPath }/mac.csv," +
-                          s"locationInfo.ActiveCountry:${ resourcesPath
-                          }/locationInfoActiveCountry.csv"
-    extColDictFilePath2 = s"deviceInformationId:${ resourcesPath }/deviceInformationId2.csv"
-    extColDictFilePath3 = s"channelsId:${ resourcesPath }/channelsId.csv"
-    header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC," +
-             "locationinfo,proddate,gamePointId,contractNumber"
-    header2 = "deviceInformationId,channelsId,contractNumber"
-  }
-
-  def buildTable() = {
-    try {
-      sql(
-        """CREATE TABLE extComplextypes (deviceInformationId int,
-     channelsId string, ROMSize string, purchasedate string,
-     mobile struct<imei:string, imsi:string>, MAC array<string>,
-     locationinfo array<struct<ActiveAreaId:int, ActiveCountry:string,
-     ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>,
-     proddate struct<productionDate:string,activeDeactivedate:array<string>>,
-     gamePointId double,contractNumber double)
-     STORED BY 'org.apache.carbondata.format'
-     TBLPROPERTIES('DICTIONARY_INCLUDE' = 'deviceInformationId, channelsId')
-        """)
-    } catch {
-      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
-    }
-
-    try {
-      sql(
-        """CREATE TABLE verticalDelimitedTable (deviceInformationId int,
-     channelsId string,contractNumber double)
-     STORED BY 'org.apache.carbondata.format'
-     TBLPROPERTIES('DICTIONARY_INCLUDE' = 'deviceInformationId, channelsId')
-        """)
-    } catch {
-      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
-    }
-
-    try {
-      sql(
-        """CREATE TABLE loadSqlTest (deviceInformationId int,
-     channelsId string, ROMSize string, purchasedate string,
-     mobile struct<imei:string, imsi:string>, MAC array<string>,
-     locationinfo array<struct<ActiveAreaId:int, ActiveCountry:string,
-     ActiveProvince:string, Activecity:string, ActiveDistrict:string, ActiveStreet:string>>,
-     proddate struct<productionDate:string,activeDeactivedate:array<string>>,
-     gamePointId double,contractNumber double)
-     STORED BY 'org.apache.carbondata.format'
-     TBLPROPERTIES('DICTIONARY_INCLUDE' = 'deviceInformationId')
-        """)
-    } catch {
-      case ex: Throwable => LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
-    }
-  }
-
-  def buildRelation() = {
-    val warehouse = s"$resourcesPath/target/warehouse"
-    val storeLocation = s"$resourcesPath/target/store"
-    val metaStoreDB = s"$resourcesPath/target"
-    CarbonProperties.getInstance()
-      .addProperty("carbon.custom.distribution", "true")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,"FORCE")
-    import org.apache.spark.sql.CarbonSession._
-
-    val spark = SparkSession
-      .builder()
-      .master("local")
-      .appName("CarbonSessionExample")
-      .config("spark.sql.warehouse.dir", warehouse)
-      .config("spark.network.timeout", "600s")
-      .config("spark.executor.heartbeatInterval", "600s")
-      .config("carbon.enable.vector.reader","false")
-      .getOrCreateCarbonSession(storeLocation, metaStoreDB)
-    val catalog = CarbonEnv.getInstance(spark).carbonMetaStore
-    extComplexRelation = catalog
-      .lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-        "extComplextypes")(spark)
-      .asInstanceOf[CarbonRelation]
-    verticalDelimiteRelation = catalog
-      .lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-        "verticalDelimitedTable")(spark)
-      .asInstanceOf[CarbonRelation]
-    loadSqlRelation = catalog.lookupRelation(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "loadSqlTest")(spark)
-      .asInstanceOf[CarbonRelation]
-  }
-
-  def buildCarbonLoadModel(
-      relation: CarbonRelation,
-      filePath: String,
-      header: String,
-      extColFilePath: String,
-      csvDelimiter: String = ","): CarbonLoadModel = {
-    val carbonLoadModel = new CarbonLoadModel
-    carbonLoadModel.setTableName(relation.carbonTable.getDatabaseName)
-    carbonLoadModel.setDatabaseName(relation.carbonTable.getTableName)
-    val table = relation.carbonTable
-    val carbonSchema = new CarbonDataLoadSchema(table)
-    carbonLoadModel.setDatabaseName(table.getDatabaseName)
-    carbonLoadModel.setTableName(table.getTableName)
-    carbonLoadModel.setCarbonTransactionalTable(table.isTransactionalTable)
-    carbonLoadModel.setTablePath(relation.carbonTable.getTablePath)
-    carbonLoadModel.setCarbonDataLoadSchema(carbonSchema)
-    carbonLoadModel.setFactFilePath(filePath)
-    carbonLoadModel.setCsvHeader(header)
-    carbonLoadModel.setCsvDelimiter(csvDelimiter)
-    carbonLoadModel.setComplexDelimiter("$")
-    carbonLoadModel.setComplexDelimiter(":")
-    carbonLoadModel.setColDictFilePath(extColFilePath)
-    carbonLoadModel.setQuoteChar("\"");
-    carbonLoadModel.setSerializationNullFormat(
-      TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + ",\\N")
-    carbonLoadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-    carbonLoadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-    carbonLoadModel.setCsvHeaderColumns(
-      LoadOption.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
-    carbonLoadModel.setMaxColumns("100")
-    // Create table and metadata folders if not exist
-    val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
-    if (!FileFactory.isFileExist(metadataDirectoryPath)) {
-      FileFactory.mkdirs(metadataDirectoryPath)
-    }
-    import scala.collection.JavaConverters._
-    val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
-      .getOrElse(CarbonCommonConstants.COMPRESSOR,
-        CompressorFactory.getInstance().getCompressor.getName)
-    carbonLoadModel.setColumnCompressor(columnCompressor)
-    carbonLoadModel
-  }
-
-  override def beforeAll {
-    cleanAllTables
-    buildTestData
-    buildTable
-    buildRelation
-  }
-
-  test("Generate global dictionary from external column file") {
-    // load the first time
-    var carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
-      header, extColDictFilePath1)
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sqlContext,
-      carbonLoadModel,
-      FileFactory.getConfiguration)
-    // check whether the dictionary is generated
-    DictionaryTestCaseUtil.checkDictionary(
-      extComplexRelation, "deviceInformationId", "10086")
-
-    // load the second time
-    carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
-      header, extColDictFilePath2)
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sqlContext,
-      carbonLoadModel,
-      FileFactory.getConfiguration)
-    // check the old dictionary and whether the new distinct value is generated
-    DictionaryTestCaseUtil.checkDictionary(
-      extComplexRelation, "deviceInformationId", "10086")
-    DictionaryTestCaseUtil.checkDictionary(
-      extComplexRelation, "deviceInformationId", "10011")
-  }
-
-  test("When csv delimiter is not comma") {
-    //  when csv delimiter is comma
-    var carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
-      header, extColDictFilePath3)
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sqlContext,
-      carbonLoadModel,
-      FileFactory.getConfiguration)
-    // check whether the dictionary is generated
-    DictionaryTestCaseUtil.checkDictionary(
-      extComplexRelation, "channelsId", "1421|")
-
-    //  when csv delimiter is not comma
-    carbonLoadModel = buildCarbonLoadModel(verticalDelimiteRelation, complexFilePath2,
-      header2, extColDictFilePath3, "|")
-    GlobalDictionaryUtil.generateGlobalDictionary(
-      sqlContext,
-      carbonLoadModel,
-      FileFactory.getConfiguration)
-    // check whether the dictionary is generated
-    DictionaryTestCaseUtil.checkDictionary(
-      verticalDelimiteRelation, "channelsId", "1431,")
-  }
-
-  test("LOAD DML with COLUMNDICT option") {
-    try {
-      sql(
-        s"""
-      LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
-      OPTIONS('FILEHEADER'='$header', 'COLUMNDICT'='$extColDictFilePath1', 'single_pass'='true')
-        """)
-    } catch {
-      case ex: Exception =>
-        LOGGER.error(ex.getMessage + "\r\n" + ex.getStackTraceString)
-        assert(false)
-    }
-    DictionaryTestCaseUtil.checkDictionary(
-      loadSqlRelation, "deviceInformationId", "10086")
-  }
-
-  test("COLUMNDICT and ALL_DICTIONARY_PATH can not be used together") {
-    val ex = intercept[MalformedCarbonCommandException] {
-      sql(
-        s"""
-        LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
-        OPTIONS('COLUMNDICT'='$extColDictFilePath1',"ALL_DICTIONARY_PATH"='$extColDictFilePath1')
-        """)
-    }
-    assertResult(ex.getMessage)(
-      "Error: COLUMNDICT and ALL_DICTIONARY_PATH can not be used together " +
-        "in options")
-  }
-
-  test("Measure can not use COLUMNDICT") {
-    val ex = intercept[DataLoadingException] {
-      sql(
-        s"""
-      LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
-      OPTIONS('single_pass'='true','FILEHEADER'='$header', 'COLUMNDICT'='gamePointId:$filePath')
-      """)
-    }
-    assertResult(ex.getMessage)(
-      "Column gamePointId is not a key column. Only key column can be part " +
-        "of dictionary and used in COLUMNDICT option.")
-  }
-
-  def cleanAllTables: Unit = {
-    sql("DROP TABLE IF EXISTS extComplextypes")
-    sql("DROP TABLE IF EXISTS verticalDelimitedTable")
-    sql("DROP TABLE IF EXISTS loadSqlTest")
-  }
-
-  override def afterAll: Unit = {
-    cleanAllTables
-  }
-}
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
index 0180384..b531fe2 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
@@ -24,6 +24,8 @@
 import org.apache.spark.sql.types._
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.util.CarbonProperties
 
 class CarbonDataSourceSuite extends Spark2QueryTest with BeforeAndAfterAll {
@@ -120,8 +122,6 @@
       .format("carbondata")
       .option("tableName", "testBigData")
       .option("tempCSV", "false")
-      .option("single_pass", "true")
-      .option("dictionary_exclude", "id") // id is high cardinality column
       .mode(SaveMode.Overwrite)
       .save()
     sql(s"select city, sum(m1) from testBigData " +
@@ -182,18 +182,15 @@
   }
 
   test("test carbon source table with string type in dictionary_exclude") {
-    try {
+    val ex = intercept[DeprecatedFeatureException] {
       sql("create table car( \nL_SHIPDATE string,\nL_SHIPMODE string,\nL_SHIPINSTRUCT string," +
           "\nL_RETURNFLAG string,\nL_RECEIPTDATE string,\nL_ORDERKEY string,\nL_PARTKEY string," +
           "\nL_SUPPKEY string,\nL_LINENUMBER int,\nL_QUANTITY decimal,\nL_EXTENDEDPRICE decimal," +
           "\nL_DISCOUNT decimal,\nL_TAX decimal,\nL_LINESTATUS string,\nL_COMMITDATE string," +
           "\nL_COMMENT string \n) \nUSING org.apache.spark.sql.CarbonSource\nOPTIONS (tableName " +
           "\"car\", DICTIONARY_EXCLUDE \"L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_COMMENT\")")
-      assert(true)
     }
-    catch {
-      case exception: Exception => assert(false)
-    }
+    assert(ex.getMessage.contains("dictionary_exclude is deprecated in CarbonData 2.0"))
   }
 
   test("test create table with complex datatype") {
@@ -264,7 +261,6 @@
          |    stringField string,
          |    intField int)
          | USING org.apache.spark.sql.CarbonSource
-         | OPTIONS('DICTIONARY_EXCLUDE'='stringField')
       """.
         stripMargin
     )
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
index 2576616..536e9ce 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
@@ -663,7 +663,7 @@
     checkAnswer(
       sql("select * from stream_table_filter where register is null"),
       Seq(Row(null, "", "", null, null, null, null, null, null)))
-    assert(1 == partitionNums("select * from stream_table_filter where register is null"))
+    assert(3 == partitionNums("select * from stream_table_filter where register is null"))
 
     checkAnswer(
       sql("select * from stream_table_filter where id is null and register is not null"),
@@ -2185,7 +2185,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='city,register', 'BAD_RECORD_PATH'='$badRecordFilePath')
+         | 'sort_columns'='name', 'BAD_RECORD_PATH'='$badRecordFilePath')
          | """.stripMargin)
 
     if (withBatchLoad) {
@@ -2214,7 +2214,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='id,name,salary,tax,percent,updated', 'BAD_RECORD_PATH'='$badRecordFilePath')
+         | 'sort_columns'='name', 'BAD_RECORD_PATH'='$badRecordFilePath')
          | """.stripMargin)
 
     if (withBatchLoad) {
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableQueryFilter.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableQueryFilter.scala
index f47e18d..a35eb16 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableQueryFilter.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableQueryFilter.scala
@@ -268,7 +268,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='name,tax,percent,updated', 'LONG_STRING_COLUMNS'='longstr')
+         | 'sort_columns'='name', 'LONG_STRING_COLUMNS'='longstr')
          | """.stripMargin)
 
     if (withBatchLoad) {
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithLongString.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithLongString.scala
index 521b241..7855d01 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithLongString.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithLongString.scala
@@ -572,7 +572,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='city,register', 'LONG_STRING_COLUMNS'='longstr')
+         | 'sort_columns'='name', 'LONG_STRING_COLUMNS'='longstr')
          | """.stripMargin)
 
     if (withBatchLoad) {
@@ -602,7 +602,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='id,name,salary,tax,percent,updated', 'LONG_STRING_COLUMNS'='longstr')
+         | 'sort_columns'='name', 'LONG_STRING_COLUMNS'='longstr')
          | """.stripMargin)
 
     if (withBatchLoad) {
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
index 1672e75..3e5f594 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableWithRowParser.scala
@@ -883,7 +883,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='city,register')
+         | 'sort_columns'='name')
          | """.stripMargin)
 
     if (withBatchLoad) {
@@ -912,7 +912,7 @@
          | )
          | STORED BY 'carbondata'
          | TBLPROPERTIES(${if (streaming) "'streaming'='true', " else "" }
-         | 'sort_columns'='name', 'dictionary_include'='id,name,salary,tax,percent,updated')
+         | 'sort_columns'='name')
          | """.stripMargin)
 
     if (withBatchLoad) {
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/commands/SetCommandTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/commands/SetCommandTestCase.scala
index 762008a..4b7fe36 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/commands/SetCommandTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/commands/SetCommandTestCase.scala
@@ -104,18 +104,6 @@
         sql(s"set ${CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE}"))
     }
   }
-  // single_pass
-  test(s"test set command for ${CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS}=true") {
-    checkAnswer(sql(s"set ${CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS}=true"),
-      sql(s"set ${CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS}"))
-  }
-
-  test(s"test set ${CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS} for invalid option") {
-    intercept[InvalidConfigurationException] {
-      checkAnswer(sql(s"set ${CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS}=123"),
-        sql(s"set ${CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS}"))
-    }
-  }
 
   test(s"test set carbon.table.load.sort.scope for valid options") {
     checkAnswer(
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/datatype/NumericDimensionBadRecordTest.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/datatype/NumericDimensionBadRecordTest.scala
index 28f2e16..e5c6b76 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/datatype/NumericDimensionBadRecordTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/datatype/NumericDimensionBadRecordTest.scala
@@ -47,39 +47,39 @@
 
       // 1. bad record int DataType dimension
       sql("create table intDataType(name String, dob timestamp, weight int)" +
-          " STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='weight')")
+          " STORED BY 'org.apache.carbondata.format' ")
       csvFilePath = s"$resourcesPath/badrecords/dummy.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO table intDataType options " +
           "('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORDS_ACTION'='IGNORE')");
       // 2. bad record long DataType dimension
       sql("create table longDataType(name String, dob timestamp, weight long)" +
-          " STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='weight')")
+          " STORED BY 'org.apache.carbondata.format' ")
       csvFilePath = s"$resourcesPath/badrecords/dummy.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO table longDataType options " +
           "('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORDS_ACTION'='IGNORE')");
       // 3. bad record double DataType dimension
       sql("create table doubleDataType(name String, dob timestamp, weight double)" +
-          " STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='weight')")
+          " STORED BY 'org.apache.carbondata.format' ")
       csvFilePath = s"$resourcesPath/badrecords/dummy.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO table doubleDataType options " +
           "('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORDS_ACTION'='IGNORE')");
 
       // 4. bad record float DataType dimension
       sql("create table floatDataType(name String, dob timestamp, weight float)" +
-          " STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='weight')")
+          " STORED BY 'org.apache.carbondata.format' ")
       csvFilePath = s"$resourcesPath/badrecords/dummy.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO table floatDataType options " +
           "('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORDS_ACTION'='IGNORE')");
       // 5. bad record decimal DataType dimension
       sql("create table bigDecimalDataType(name String, dob timestamp, weight decimal(3,1))" +
-          " STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='weight')")
+          " STORED BY 'org.apache.carbondata.format' ")
       csvFilePath = s"$resourcesPath/badrecords/dummy.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO table bigDecimalDataType options " +
           "('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORDS_ACTION'='IGNORE')");
 
       // 6. bad record string DataType dimension
       sql("create table stringDataType(name String, dob timestamp, weight String)" +
-          " STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='weight')")
+          " STORED BY 'org.apache.carbondata.format' ")
       csvFilePath = s"$resourcesPath/badrecords/dummy.csv"
       sql("LOAD DATA local inpath '" + csvFilePath + "' INTO table stringDataType options " +
           "('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORDS_ACTION'='IGNORE')");
@@ -100,8 +100,7 @@
         "row format delimited fields terminated by ','")
     sql("""insert into num_dic select 'sam','\N'""")
     sql("select * from num_dic").show()
-    sql("create table num_dicc(cust_name string, cust_id int) stored by 'carbondata'" +
-              " TBLPROPERTIES('DICTIONARY_INCLUDE'='cust_id')")
+    sql("create table num_dicc(cust_name string, cust_id int) stored by 'carbondata'")
     try {
       sql("insert into table num_dicc select * from num_dic")
     } catch {
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryJoinTestSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryJoinTestSuite.scala
index 4552b4f..51e4459 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryJoinTestSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/SubQueryJoinTestSuite.scala
@@ -27,7 +27,7 @@
     sql("drop table if exists t1")
     sql("drop table if exists t2")
 
-    sql("create table t1 (s1 string) stored by 'carbondata' tblproperties('dictionary_include'='s1')")
+    sql("create table t1 (s1 string) stored by 'carbondata' ")
     sql("insert into t1 select 'abcd' ")
     sql("insert into t1 select 'efgh' ")
     sql("insert into t1 select 'ijkl' ")
@@ -63,11 +63,8 @@
     sql(
       "create table t1 (m_month smallint, hs_code string, country smallint, dollar_value double, " +
       "quantity double, unit smallint, b_country smallint, imex int, y_year smallint) stored by " +
-      "'carbondata' tblproperties('dictionary_include'='m_month,hs_code,b_country,unit,y_year," +
-      "imex', 'sort_columns'='y_year,m_month,country,b_country,imex')")
-    sql(
-      "create table t2(id bigint, hs string, hs_cn string, hs_en string) stored by 'carbondata' " +
-      "tblproperties ('dictionary_include'='id,hs,hs_cn,hs_en')")
+      "'carbondata' tblproperties('sort_columns'='y_year,m_month,country,b_country,imex')")
+    sql("create table t2(id bigint, hs string, hs_cn string, hs_en string) stored by 'carbondata' ")
     checkAnswer(sql(
       "select a.hs,count(*) tb from (select substring(hs_code,1,2) as hs,count(*) v2000 from t1 " +
       "group by substring(hs_code,1,2),y_year) a left join t2 h on (a.hs=h.hs) group by a.hs"),
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala
index d6c1d20..4d93539 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/query/TestNotEqualToFilter.scala
@@ -40,7 +40,7 @@
         CREATE TABLE IF NOT EXISTS test_not_equal_to_carbon
         (ID Int, date Timestamp, country String,
         name String, phonetype String, serialname String, salary Int)
-        STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+        STORED BY 'org.apache.carbondata.format'
       """)
     sql(
       """
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala
index 0b0ac75..1a54ef3 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/register/TestRegisterCarbonTable.scala
@@ -170,7 +170,7 @@
       restoreData(dblocation, "carbontable")
       sql("refresh table carbontable")
       sql("Alter table carbontable add columns(c4 string) " +
-          "TBLPROPERTIES('DICTIONARY_EXCLUDE'='c4', 'DEFAULT.VALUE.c4'='def')")
+          "TBLPROPERTIES('DEFAULT.VALUE.c4'='def')")
       checkAnswer(
         sql("""select c1,c2,c3,c5,c4 from carbon.carbontable"""),
         Seq(Row("a", 1, "aa", "aaa", "def"), Row("b", 1, "bb", "bbb", "def"))
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index dce7c73..7c7bcc8 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -32,7 +32,8 @@
 class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {
-
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
     sql("drop table if exists restructure")
     sql("drop table if exists table1")
     sql("drop table if exists restructure_test")
@@ -82,21 +83,21 @@
 
   test("test add dictionary column") {
     sql(
-      "alter table restructure add columns(dict int) TBLPROPERTIES ('DICTIONARY_INCLUDE'='dict', " +
+      "alter table restructure add columns(dict int) TBLPROPERTIES ( " +
       "'DEFAULT.VALUE.dict'= '9999')")
     checkAnswer(sql("select distinct(dict) from restructure"), Row(9999))
   }
   test("test add no dictionary column") {
     sql(
       "alter table restructure add columns(nodict string) TBLPROPERTIES " +
-      "('DICTIONARY_EXCLUDE'='nodict', 'DEFAULT.VALUE.NoDict'= 'abcd')")
+      "('DEFAULT.VALUE.NoDict'= 'abcd')")
     checkAnswer(sql("select distinct(nodict) from restructure"), Row("abcd"))
   }
 
   ignore ("test add timestamp no dictionary column") {
     sql(
       "alter table restructure add columns(tmpstmp timestamp) TBLPROPERTIES ('DEFAULT.VALUE" +
-      ".tmpstmp'= '17-01-2007')")
+      ".tmpstmp'= '2007-01-17')")
     sql("select tmpstmp from restructure").show(200,false)
     sql("select distinct(tmpstmp) from restructure").show(200,false)
     checkAnswer(sql("select distinct(tmpstmp) from restructure"),
@@ -107,7 +108,7 @@
   test ("test add timestamp direct dictionary column") {
     sql(
       "alter table restructure add columns(tmpstmp1 timestamp) TBLPROPERTIES ('DEFAULT.VALUE" +
-      ".tmpstmp1'= '17-01-3007','DICTIONARY_INCLUDE'='tmpstmp1')")
+      ".tmpstmp1'= '3007-01-17')")
     checkAnswer(sql("select distinct(tmpstmp1) from restructure"),
       Row(null))
     checkExistence(sql("desc restructure"), true, "tmpstmp", "timestamp")
@@ -117,7 +118,7 @@
     sql("create table table1(name string) stored by 'carbondata'")
     sql("insert into table1 select 'abc'")
     sql("alter table table1 add columns(tmpstmp timestamp) TBLPROPERTIES " +
-        "('DEFAULT.VALUE.tmpstmp'='17-01-3007','DICTIONARY_INCLUDE'= 'tmpstmp')")
+        "('DEFAULT.VALUE.tmpstmp'='17-01-3007')")
     sql("insert into table1 select 'name','17-01-2007'")
     checkAnswer(sql("select * from table1"),
       Seq(Row("abc",null),
@@ -163,8 +164,7 @@
       "alter table restructure add columns(strfld string, datefld date, tptfld timestamp, " +
       "shortFld smallInt, " +
       "intFld int, longFld bigint, dblFld double,dcml decimal(5,4))TBLPROPERTIES" +
-      "('DICTIONARY_INCLUDE'='datefld,shortFld,intFld,longFld,dblFld,dcml', 'DEFAULT.VALUE" +
-      ".dblFld'= '12345')")
+      "('DEFAULT.VALUE.dblFld'= '12345')")
     checkAnswer(sql("select distinct(dblFld) from restructure"),
       Row(java.lang.Double.parseDouble("12345")))
     checkExistence(sql("desc restructure"), true, "strfld", "string")
@@ -224,8 +224,7 @@
     sql("alter table restructure add columns(dimfld1 string, msrCol double)")
     try {
       sql(
-        "alter table restructure add columns(dimfld1 int)TBLPROPERTIES" +
-        "('DICTIONARY_INCLUDE'='dimfld1')")
+        "alter table restructure add columns(dimfld1 int)")
       sys.error("Exception should be thrown as dimfld1 is already exist")
     } catch {
       case e: Exception =>
@@ -242,9 +241,7 @@
 
   test("test adding no dictionary column with numeric type") {
     try {
-      sql(
-        "alter table restructure add columns(dimfld2 double) TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='dimfld2')")
+      sql("alter table restructure add columns(dimfld2 double)")
       sys.error("Exception should be thrown as msrCol is already exist")
     } catch {
       case e: Exception =>
@@ -266,7 +263,7 @@
     sql("alter table restructure drop columns(empname)")
     sql(
       "alter table restructure add columns(empname int) TBLPROPERTIES" +
-      "('DICTIONARY_INCLUDE'='empname', 'DEFAULT.VALUE.empname'='12345')")
+      "('DEFAULT.VALUE.empname'='12345')")
     checkAnswer(sql("select distinct(empname) from restructure"), Row(12345))
     checkAnswer(sql("select count(empname) from restructure"), Row(10))
   }
@@ -282,7 +279,7 @@
     }
     sql(
       "alter table restructure add columns(empname string) TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='empname', 'DEFAULT.VALUE.empname'='testuser')")
+      "('DEFAULT.VALUE.empname'='testuser')")
     checkAnswer(sql("select distinct(empname) from restructure"), Row("testuser"))
     checkAnswer(sql("select count(empname) from restructure"), Row(10))
   }
@@ -336,13 +333,13 @@
     sql("alter table restructure drop columns(designation)")
     sql(
       "alter table default.restructure add columns(designation int) TBLPROPERTIES" +
-      "('DICTIONARY_INCLUDE'='designation', 'DEFAULT.VALUE.designation'='12345')")
+      "('DEFAULT.VALUE.designation'='12345')")
     checkAnswer(sql("select distinct(designation) from restructure"), Row(12345))
     // drop and add nodict column
     sql("alter table restructure drop columns(designation)")
     sql(
       "alter table restructure add columns(designation string) TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='designation', 'DEFAULT.VALUE.designation'='abcd')")
+      "('DEFAULT.VALUE.designation'='abcd')")
     checkAnswer(sql("select distinct(designation) from restructure"), Row("abcd"))
     // drop and add directdict column
     sql("alter table restructure drop columns(designation)")
@@ -525,15 +522,14 @@
 
   // after changing default sort_scope to no_sort, all dimensions are not selected for sorting.
   ignore("describe formatted for default sort_columns pre and post alter") {
-    sql("CREATE TABLE defaultSortColumnsWithAlter (empno int, empname String, designation String,role String, doj Timestamp) STORED BY 'org.apache.carbondata.format' " +
-        "tblproperties('DICTIONARY_INCLUDE'='empno','DICTIONARY_EXCLUDE'='role')")
+    sql("CREATE TABLE defaultSortColumnsWithAlter (empno int, empname String, designation String,role String, doj Timestamp) STORED BY 'org.apache.carbondata.format' ")
     sql("alter table defaultSortColumnsWithAlter drop columns (designation)")
     sql("alter table defaultSortColumnsWithAlter add columns (designation12 String)")
     checkExistence(sql("describe formatted defaultSortColumnsWithAlter"),true,"Sort Columns empno, empname, role, doj")
   }
   test("describe formatted for specified sort_columns pre and post alter") {
     sql("CREATE TABLE specifiedSortColumnsWithAlter (empno int, empname String, designation String,role String, doj Timestamp) STORED BY 'org.apache.carbondata.format' " +
-        "tblproperties('sort_columns'='empno,empname,designation,role,doj','DICTIONARY_INCLUDE'='empno','DICTIONARY_EXCLUDE'='role')")
+        "tblproperties('sort_columns'='empno,empname,designation,role,doj')")
     sql("alter table specifiedSortColumnsWithAlter drop columns (designation)")
     sql("alter table specifiedSortColumnsWithAlter add columns (designation12 String)")
     checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"Sort Columns empno, empname, role, doj")
@@ -554,7 +550,7 @@
     sql("ALTER TABLE restructure1 ADD COLUMNS (projId int)")
     sql("ALTER TABLE restructure1 DROP COLUMNS (projId)")
     sql("ALTER TABLE restructure1 CHANGE empno empno BIGINT")
-    sql("ALTER TABLE restructure1 ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_EXCLUDE'='b1')")
+    sql("ALTER TABLE restructure1 ADD COLUMNS (a1 INT, b1 STRING)")
     checkAnswer(sql("select a1,b1,empname from restructure1 where a1 is null and b1 is null and empname='arvind'"),Row(null,null,"arvind"))
     sql("drop table if exists restructure1")
     sql("drop table if exists restructure")
@@ -608,7 +604,7 @@
         if (flag) {
           sql(
             "alter table test add columns (c3 timestamp) TBLPROPERTIES('DEFAULT.VALUE.c3' = " +
-            "'1996/01/01 11:11:11', 'DICTIONARY_INCLUDE' = 'c3')")
+            "'1996/01/01 11:11:11')")
         } else {
           sql(
             "alter table test add columns (c3 timestamp) TBLPROPERTIES('DEFAULT.VALUE.c3' = " +
@@ -652,8 +648,7 @@
         "string,longdate date) stored by 'carbondata'")
       sql("insert into test select 1,'String1',12345,'area',5000,'country','2017/02/12'")
       if (flag) {
-        sql(s"alter table test add columns (c3 int) TBLPROPERTIES('DEFAULT.VALUE.c3' = '23', " +
-            s"'DICTIONARY_INCLUDE'='c3')")
+        sql(s"alter table test add columns (c3 int) TBLPROPERTIES('DEFAULT.VALUE.c3' = '23')")
       } else {
         sql(s"alter table test add columns (c3 int) TBLPROPERTIES('DEFAULT.VALUE.c3' = '23')")
       }
@@ -683,9 +678,9 @@
     val testData = s"$resourcesPath/sample.csv"
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table retructure_iud")
     sql("ALTER TABLE retructure_iud ADD COLUMNS (newField STRING) " +
-        "TBLPROPERTIES ('DEFAULT.VALUE.newField'='def', 'DICTIONARY_INCLUDE'='newField')").show()
+        "TBLPROPERTIES ('DEFAULT.VALUE.newField'='def')").show()
     sql("ALTER TABLE retructure_iud ADD COLUMNS (newField1 STRING) " +
-        "TBLPROPERTIES ('DEFAULT.VALUE.newField1'='def', 'DICTIONARY_EXCLUDE'='newField1')").show()
+        "TBLPROPERTIES ('DEFAULT.VALUE.newField1'='def')").show()
     // update operation
     sql("""update retructure_iud d  set (d.id) = (d.id + 1) where d.id > 2""").show()
     checkAnswer(
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index fd15e89..ca2c06b 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -44,7 +44,7 @@
         s"OPTIONS('FILEHEADER'='intField,stringField,timestampField,decimalField')")
     sql(
       "ALTER TABLE addcolumntest ADD COLUMNS(charField STRING) TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField', 'DEFAULT.VALUE.charfield'='def')")
+      "('DEFAULT.VALUE.charfield'='def')")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE addcolumntest " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql("CREATE TABLE hivetable STORED AS PARQUET SELECT * FROM addcolumntest")
@@ -89,7 +89,7 @@
     def test_add_and_filter() = {
       sql(
         "ALTER TABLE addcolumntest ADD COLUMNS(intnewField INT) TBLPROPERTIES" +
-          "('DICTIONARY_INCLUDE'='intnewField', 'DEFAULT.VALUE.intNewField'='5')")
+          "('DEFAULT.VALUE.intNewField'='5')")
       checkAnswer(sql("SELECT charField FROM addcolumntest WHERE intnewField > 2"),
         Seq(Row("abc"), Row("def")))
       checkAnswer(sql("SELECT charField FROM addcolumntest WHERE intnewField < 2"), Seq())
@@ -152,8 +152,7 @@
       sql("DROP TABLE IF EXISTS carbon_table")
       sql(
         "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-        "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='charField')")
+        "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' ")
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
           s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
       sql("ALTER TABLE carbon_table DROP COLUMNS(timestampField)")
@@ -176,17 +175,14 @@
       sql("DROP TABLE IF EXISTS carbon_table")
       sql(
         "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-        "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='charField')")
+        "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' ")
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
           s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
       sql("ALTER TABLE carbon_table DROP COLUMNS(charField)")
       sql("SELECT * FROM carbon_table").collect
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data4.csv' INTO TABLE carbon_table " +
           s"OPTIONS('FILEHEADER'='intField,stringField,timestampField,decimalField')")
-      sql(
-        "ALTER TABLE carbon_table ADD COLUMNS(charField STRING) TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='charField')")
+      sql("ALTER TABLE carbon_table ADD COLUMNS(charField STRING) ")
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data2.csv' INTO TABLE carbon_table " +
           s"OPTIONS('FILEHEADER'='intField,stringField,timestampField,decimalField,charField')")
       sql("SELECT * FROM carbon_table").collect
@@ -209,8 +205,7 @@
     sql("DROP TABLE IF EXISTS carbon_table")
     sql(
       "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-      "TIMESTAMP)STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField')")
+      "TIMESTAMP)STORED BY 'carbondata' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
@@ -231,8 +226,7 @@
     sql("DROP TABLE IF EXISTS carbon_table")
     sql(
       "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-      "TIMESTAMP)STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField')")
+      "TIMESTAMP)STORED BY 'carbondata'")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql("ALTER TABLE carbon_table ADD COLUMNS(newfield char(10)) TBLPROPERTIES ('DEFAULT.VALUE.newfield'='char')")
@@ -246,9 +240,7 @@
       sql("DROP TABLE IF EXISTS carbon_table")
       sql(
         "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-
-        "TIMESTAMP)STORED BY 'carbondata' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='charField')")
+        "TIMESTAMP)STORED BY 'carbondata' ")
       sql(
         "ALTER TABLE carbon_table ADD COLUMNS(newfield char) TBLPROPERTIES ('DEFAULT.VALUE.newfield'='c')")
       sql("DROP TABLE IF EXISTS carbon_table")
@@ -264,8 +256,7 @@
     sql("DROP TABLE IF EXISTS carbon_table")
     sql(
       "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-      "TIMESTAMP)STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField')")
+      "TIMESTAMP)STORED BY 'carbondata' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql("ALTER TABLE carbon_table ADD COLUMNS(newfield varchar(10)) TBLPROPERTIES ('DEFAULT.VALUE.newfield'='char')")
@@ -280,8 +271,7 @@
       sql(
         "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
 
-        "TIMESTAMP)STORED BY 'carbondata' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='charField')")
+        "TIMESTAMP)STORED BY 'carbondata' ")
       sql(
         "ALTER TABLE carbon_table ADD COLUMNS(newfield varchar) TBLPROPERTIES ('DEFAULT.VALUE.newfield'='c')")
       sql("DROP TABLE IF EXISTS carbon_table")
@@ -298,8 +288,7 @@
       sql("DROP TABLE IF EXISTS carbon_table")
       sql(
         "CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField " +
-        "TIMESTAMP)STORED BY 'carbondata' TBLPROPERTIES" +
-        "('DICTIONARY_EXCLUDE'='charField')")
+        "TIMESTAMP)STORED BY 'carbondata' ")
       val lockFilePath = s"${ TestQueryExecutor.storeLocation }/default/carbon_table/meta.lock"
       new File(lockFilePath).createNewFile()
       sql(
@@ -317,13 +306,12 @@
     sql("DROP TABLE IF EXISTS carbon_new")
     sql(
       "CREATE TABLE carbon_new(intField INT,stringField STRING,charField STRING,timestampField " +
-      "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField')")
+      "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_new " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql(
       "ALTER TABLE carbon_new ADD COLUMNS(newField STRING) TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='newField','DEFAULT.VALUE.newField'='def')")
+      "('DEFAULT.VALUE.newField'='def')")
     checkAnswer(sql("SELECT * FROM carbon_new LIMIT 1"),
       Row(new Integer(100),
         "spark",
@@ -339,13 +327,11 @@
     sql("DROP TABLE IF EXISTS carbon_new")
     sql(
       "CREATE TABLE carbon_new(intField INT,stringField STRING,charField STRING,timestampField " +
-      "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField')")
+      "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_new " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql(
-      "ALTER TABLE carbon_new ADD COLUMNS(newField STRING) TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='newField')")
+      "ALTER TABLE carbon_new ADD COLUMNS(newField STRING) ")
     assert(sql(
       "SELECT intField,stringField,charField,timestampField,decimalField, newField FROM " +
       "carbon_new LIMIT 1").count().equals(1L))
@@ -357,13 +343,12 @@
     sql("DROP TABLE IF EXISTS carbon_new")
     sql(
       "CREATE TABLE carbon_new(intField INT,stringField STRING,charField STRING,timestampField " +
-      "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='charField')")
+      "TIMESTAMP,decimalField DECIMAL(6,2))STORED BY 'carbondata' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_new " +
         s"OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql(
       "ALTER TABLE carbon_new ADD COLUMNS(newField STRING) TBLPROPERTIES" +
-      "('DICTIONARY_EXCLUDE'='newField','DEFAULT.VALUE.newField'='def')")
+      "('DEFAULT.VALUE.newField'='def')")
     checkAnswer(sql(
       "SELECT intField,stringField,charField,newField,timestampField,decimalField FROM " +
       "carbon_new LIMIT 1"), Row(new Integer(100),
@@ -392,7 +377,7 @@
         "INTEGER_COLUMN1')")
     sql(
       """ALTER TABLE carbon_new ADD COLUMNS(CUST_NAME STRING) TBLPROPERTIES
-        ('DICTIONARY_EXCLUDE'='CUST_NAME', 'DEFAULT.VALUE.CUST_NAME'='testuser')""")
+        ('DEFAULT.VALUE.CUST_NAME'='testuser')""")
     checkAnswer(sql("SELECT DISTINCT(CUST_NAME) FROM carbon_new"),Row("testuser"))
   }
 
@@ -436,7 +421,7 @@
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql(
       "ALTER TABLE carbon_table ADD COLUMNS(newField INT) TBLPROPERTIES" +
-      "('DEFAULT.VALUE.newField'='67890', 'DICTIONARY_EXCLUDE'='newField')")
+      "('DEFAULT.VALUE.newField'='67890')")
     checkAnswer(sql("SELECT DISTINCT(newField) FROM carbon_table"), Row(67890))
     sql("DROP TABLE IF EXISTS carbon_table")
   }
@@ -448,7 +433,7 @@
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
     sql(
       "ALTER TABLE carbon_table ADD COLUMNS(newField bigint) TBLPROPERTIES" +
-      "('DEFAULT.VALUE.newField'='67890', 'DICTIONARY_EXCLUDE'='newField')")
+      "('DEFAULT.VALUE.newField'='67890')")
     checkAnswer(sql("SELECT DISTINCT(newField) FROM carbon_table"), Row(67890))
     sql("DROP TABLE IF EXISTS carbon_table")
   }
@@ -502,8 +487,7 @@
       s"('BAD_RECORDS_LOGGER_ENABLE'='TRUE', " +
       s"'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID,CUST_NAME')")
     sql(
-      "ALTER TABLE carbon_dictionary_is_null ADD COLUMNS (a6 INT) TBLPROPERTIES" +
-      "('dictionary_include'='a6')")
+      "ALTER TABLE carbon_dictionary_is_null ADD COLUMNS (a6 INT) ")
     sql(
       s"LOAD DATA INPATH '$resourcesPath/restructure/data6.csv' INTO TABLE " +
       s"carbon_dictionary_is_null" +
@@ -586,7 +570,7 @@
   test("test compaction with all no dictionary columns") {
     sqlContext.setConf("carbon.enable.vector.reader", "true")
     sql("DROP TABLE IF EXISTS alter_no_dict")
-    sql("CREATE TABLE alter_no_dict(stringField STRING,charField STRING) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='stringField,charField')")
+    sql("CREATE TABLE alter_no_dict(stringField STRING,charField STRING) STORED BY 'carbondata' ")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data7.csv' INTO TABLE alter_no_dict OPTIONS('FILEHEADER'='stringField,charField')")
     sql("ALTER TABLE alter_no_dict DROP COLUMNS(charField)")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data7.csv' INTO TABLE alter_no_dict OPTIONS('FILEHEADER'='stringField')")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
index 4f3a0cf..a76bef5 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
@@ -153,7 +153,7 @@
       "CREATE TABLE rename (empno int, empname String, designation String, doj Timestamp, " +
       "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
-      "utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties('dictionary_include'='empno,empname'," +
+      "utilization int,salary int) STORED BY 'org.apache.carbondata.format' tblproperties(" +
       "'local_dictionary_include'='workgroupcategoryname','local_dictionary_exclude'='deptname','COLUMN_META_CACHE'='projectcode,attendance'," +
       "'SORT_COLUMNS'='workgroupcategory,utilization,salary')")
     sql("alter table rename change eMPName name string")
@@ -163,10 +163,6 @@
     sql("alter table rename change uTiLIZation utILIty int")
 
     val descLoc = sql("describe formatted rename").collect
-    descLoc.find(_.get(0).toString.contains("Global Dictionary")) match {
-      case Some(row) => assert(row.get(1).toString.contains("empno,name"))
-      case None => assert(false)
-    }
     descLoc.find(_.get(0).toString.contains("Local Dictionary Include")) match {
       case Some(row) => assert(row.get(1).toString.contains("workgroup"))
       case None => assert(false)
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
index eff7593..b26ddfa 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
@@ -153,7 +153,7 @@
     def test_change_data_type() = {
       beforeAll
     sql("drop table if exists table_sort")
-    sql("CREATE TABLE table_sort (imei int,age int,mac string) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='imei,age','SORT_COLUMNS'='imei,age')")
+    sql("CREATE TABLE table_sort (imei int,age int,mac string) STORED BY 'carbondata' TBLPROPERTIES('SORT_COLUMNS'='imei,age')")
     sql("insert into table_sort select 32674,32794,'MAC1'")
     sql("alter table table_sort change age age bigint")
     sql("insert into table_sort select 32675,9223372036854775807,'MAC2'")
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 557c490..b7a4508 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
@@ -25,26 +25,16 @@
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-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.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.datastore.row.ComplexColumnInfo;
 import org.apache.carbondata.core.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -54,9 +44,7 @@
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
 import org.apache.carbondata.processing.loading.dictionary.DirectDictionary;
-import org.apache.carbondata.processing.loading.dictionary.PreCreatedDictionary;
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -146,16 +134,10 @@
    * @param parentName
    * @param columnId
    * @param carbonDimension
-   * @param absoluteTableIdentifier
-   * @param client
-   * @param useOnePass
-   * @param localCache
    * @param nullFormat
    */
   public PrimitiveDataType(CarbonColumn carbonColumn, String parentName, String columnId,
-      CarbonDimension carbonDimension, AbsoluteTableIdentifier absoluteTableIdentifier,
-      DictionaryClient client, Boolean useOnePass, Map<Object, Integer> localCache,
-      String nullFormat) {
+      CarbonDimension carbonDimension, String nullFormat) {
     this.name = carbonColumn.getColName();
     this.parentName = parentName;
     this.columnId = columnId;
@@ -164,42 +146,12 @@
     this.nullFormat = nullFormat;
     this.dataType = carbonColumn.getDataType();
 
-    DictionaryColumnUniqueIdentifier identifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType());
-    try {
-      if (carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
-          || carbonColumn.getDataType() == DataTypes.DATE) {
-        dictionaryGenerator = new DirectDictionary(DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(carbonDimension.getDataType(),
-                getDateFormat(carbonDimension)));
-        isDirectDictionary = true;
-      } else if (carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
-        CacheProvider cacheProvider = CacheProvider.getInstance();
-        Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache =
-            cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
-        Dictionary dictionary = null;
-        if (useOnePass) {
-          if (CarbonUtil.isFileExistsForGivenColumn(identifier)) {
-            dictionary = cache.get(identifier);
-          }
-          DictionaryMessage dictionaryMessage = new DictionaryMessage();
-          dictionaryMessage.setColumnName(carbonDimension.getColName());
-          // for table initialization
-          dictionaryMessage
-              .setTableUniqueId(absoluteTableIdentifier.getCarbonTableIdentifier().getTableId());
-          dictionaryMessage.setData("0");
-          // for generate dictionary
-          dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
-          dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,
-              dictionaryMessage, localCache);
-        } else {
-          dictionary = cache.get(identifier);
-          dictionaryGenerator = new PreCreatedDictionary(dictionary);
-        }
-      }
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+    if (carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
+        || carbonColumn.getDataType() == DataTypes.DATE) {
+      dictionaryGenerator = new DirectDictionary(DirectDictionaryKeyGeneratorFactory
+          .getDirectDictionaryGenerator(carbonDimension.getDataType(),
+              getDateFormat(carbonDimension)));
+      isDirectDictionary = true;
     }
   }
 
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
index ed6dd4e..54e7bf7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
@@ -23,7 +23,6 @@
 import java.util.Map;
 
 import org.apache.carbondata.core.datastore.TableSpec;
-import org.apache.carbondata.core.dictionary.service.DictionaryServiceProvider;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -51,36 +50,6 @@
 
   private Map<String, Object> dataLoadProperties = new HashMap<>();
 
-  /**
-   *  Use one pass to generate dictionary
-   */
-  private boolean useOnePass;
-
-  /**
-   * dictionary server host
-   */
-  private String dictionaryServerHost;
-
-  /**
-   * dictionary sever port
-   */
-  private int dictionaryServerPort;
-
-  /**
-   * dictionary server secret key
-   */
-  private String dictionaryServerSecretKey;
-
-  /**
-   * Dictionary Service Provider.
-   */
-  private DictionaryServiceProvider dictionaryServiceProvider;
-
-  /**
-   * Secure Mode or not.
-   */
-  private Boolean dictionaryEncryptServerSecure;
-
   private boolean preFetch;
 
   private int dimensionCount;
@@ -256,54 +225,6 @@
     this.bucketingInfo = bucketingInfo;
   }
 
-  public boolean getUseOnePass() {
-    return useOnePass;
-  }
-
-  public void setUseOnePass(boolean useOnePass) {
-    this.useOnePass = useOnePass;
-  }
-
-  public String getDictionaryServerHost() {
-    return dictionaryServerHost;
-  }
-
-  public void setDictionaryServerHost(String dictionaryServerHost) {
-    this.dictionaryServerHost = dictionaryServerHost;
-  }
-
-  public int getDictionaryServerPort() {
-    return dictionaryServerPort;
-  }
-
-  public void setDictionaryServerPort(int dictionaryServerPort) {
-    this.dictionaryServerPort = dictionaryServerPort;
-  }
-
-  public String getDictionaryServerSecretKey() {
-    return dictionaryServerSecretKey;
-  }
-
-  public void setDictionaryServerSecretKey(String dictionaryServerSecretKey) {
-    this.dictionaryServerSecretKey = dictionaryServerSecretKey;
-  }
-
-  public DictionaryServiceProvider getDictionaryServiceProvider() {
-    return dictionaryServiceProvider;
-  }
-
-  public void setDictionaryServiceProvider(DictionaryServiceProvider dictionaryServiceProvider) {
-    this.dictionaryServiceProvider = dictionaryServiceProvider;
-  }
-
-  public Boolean getDictionaryEncryptServerSecure() {
-    return dictionaryEncryptServerSecure;
-  }
-
-  public void setDictionaryEncryptServerSecure(Boolean dictionaryEncryptServerSecure) {
-    this.dictionaryEncryptServerSecure = dictionaryEncryptServerSecure;
-  }
-
   public boolean isPreFetch() {
     return preFetch;
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index 7c7834e..20fe32e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -271,13 +271,6 @@
     configuration.setDataFields(
         updateDataFieldsBasedOnSortColumns(dataFields).toArray(new DataField[dataFields.size()]));
     configuration.setBucketingInfo(carbonTable.getBucketingInfo());
-    // configuration for one pass load: dictionary server info
-    configuration.setUseOnePass(loadModel.getUseOnePass());
-    configuration.setDictionaryServerHost(loadModel.getDictionaryServerHost());
-    configuration.setDictionaryServerPort(loadModel.getDictionaryServerPort());
-    configuration.setDictionaryServerSecretKey(loadModel.getDictionaryServerSecretKey());
-    configuration.setDictionaryEncryptServerSecure(loadModel.getDictionaryEncryptServerSecure());
-    configuration.setDictionaryServiceProvider(loadModel.getDictionaryServiceProvider());
     configuration.setPreFetch(loadModel.isPreFetch());
     configuration.setNumberOfSortColumns(carbonTable.getNumberOfSortColumns());
     configuration.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns());
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
deleted file mode 100644
index 2f6478c..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.converter.impl;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.Cache;
-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.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-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;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
-import org.apache.carbondata.processing.loading.dictionary.PreCreatedDictionary;
-import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
-
-  private BiDictionary<Integer, Object> dictionaryGenerator;
-
-  private int index;
-
-  private CarbonDimension carbonDimension;
-
-  private String nullFormat;
-
-  private Dictionary dictionary;
-
-  private DictionaryMessage dictionaryMessage;
-
-  private boolean isEmptyBadRecord;
-
-  public DictionaryFieldConverterImpl(CarbonColumn carbonColumn,
-      String tableId, String nullFormat, int index,
-      DictionaryClient client, boolean useOnePass, Map<Object, Integer> localCache,
-      boolean isEmptyBadRecord, DictionaryColumnUniqueIdentifier identifier) throws IOException {
-    this.index = index;
-    assert carbonColumn instanceof CarbonDimension;
-    this.carbonDimension = (CarbonDimension) carbonColumn;
-    this.nullFormat = nullFormat;
-    this.isEmptyBadRecord = isEmptyBadRecord;
-
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache =
-        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
-
-    // if use one pass, use DictionaryServerClientDictionary
-    if (useOnePass) {
-      if (CarbonUtil.isFileExistsForGivenColumn(identifier)) {
-        dictionary = cache.get(identifier);
-      }
-      dictionaryMessage = new DictionaryMessage();
-      dictionaryMessage.setColumnName(carbonColumn.getColName());
-      // for table initialization
-      dictionaryMessage.setTableUniqueId(tableId);
-      dictionaryMessage.setData("0");
-      // for generate dictionary
-      dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
-      dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,
-          dictionaryMessage, localCache);
-    } else {
-      dictionary = cache.get(identifier);
-      dictionaryGenerator = new PreCreatedDictionary(dictionary);
-    }
-  }
-
-  @Override
-  public void convert(CarbonRow row, BadRecordLogHolder logHolder)
-      throws CarbonDataLoadingException {
-    try {
-      row.update(convert(row.getString(index), logHolder), index);
-    } catch (RuntimeException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  @Override
-  public Object convert(Object value, BadRecordLogHolder logHolder)
-      throws RuntimeException {
-    try {
-      String parsedValue = null;
-      String dimensionValue = (String) value;
-      if (dimensionValue == null || dimensionValue.equals(nullFormat)) {
-        parsedValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
-      } else {
-        parsedValue = DataTypeUtil.parseValue(dimensionValue, carbonDimension);
-      }
-      if (null == parsedValue) {
-        if ((dimensionValue.length() > 0) || (dimensionValue.length() == 0 && isEmptyBadRecord)) {
-          String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
-          if (null == message) {
-            message = CarbonDataProcessorUtil
-                .prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
-            logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
-          }
-          logHolder.setReason(message);
-        }
-        return CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
-      } else {
-        return dictionaryGenerator.getOrGenerateKey(parsedValue);
-      }
-    } catch (DictionaryGenerationException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Method to clear out the dictionary cache.
-   */
-  @Override
-  public void clear() {
-    CarbonUtil.clearDictionaryCache(dictionary);
-  }
-
-  @Override
-  public void fillColumnCardinality(List<Integer> cardinality) {
-    cardinality.add(dictionaryGenerator.size());
-  }
-
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index a59df4d..0b43a5e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -17,23 +17,14 @@
 
 package org.apache.carbondata.processing.loading.converter.impl;
 
-import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
@@ -67,95 +58,25 @@
   /**
    * Creates the FieldConverter for all dimensions, for measures return null.
    *
-   * @param dataField                 column schema
-   * @param absoluteTableIdentifier   table identifier
-   * @param index                     index of column in the row
-   * @param nullFormat                null format of the field
-   * @param client
-   * @param useOnePass
-   * @param localCache
-   * @param isEmptyBadRecord
-   * @param parentTablePath
-   * @param isConvertToBinary     whether the no dictionary field to be converted to binary or not
-   * @return
-   * @throws IOException
-   */
-  public FieldConverter createFieldEncoder(DataField dataField,
-      AbsoluteTableIdentifier absoluteTableIdentifier, int index, String nullFormat,
-      DictionaryClient client, Boolean useOnePass, Map<Object, Integer> localCache,
-      boolean isEmptyBadRecord, String parentTablePath, boolean isConvertToBinary)
-      throws IOException {
-    return createFieldEncoder(dataField, absoluteTableIdentifier, index, nullFormat, client,
-        useOnePass, localCache, isEmptyBadRecord, parentTablePath, isConvertToBinary,
-        CarbonLoadOptionConstants.CARBON_OPTIONS_BINARY_DECODER_DEFAULT);
-  }
-
-  /**
-   * Creates the FieldConverter for all dimensions, for measures return null.
-   *
    * @param dataField               column schema
-   * @param absoluteTableIdentifier table identifier
    * @param index                   index of column in the row
    * @param nullFormat              null format of the field
-   * @param client                  Dictionary Client
-   * @param useOnePass              whether use OnePass
-   * @param localCache              local Cache
    * @param isEmptyBadRecord        whether is Empty BadRecord
-   * @param parentTablePath         parent tabel path
    * @param isConvertToBinary       whether the no dictionary field to be converted to binary or not
    * @param binaryDecoder           carbon binary decoder for loading data
    * @return
-   * @throws IOException
    */
-  public FieldConverter createFieldEncoder(DataField dataField,
-      AbsoluteTableIdentifier absoluteTableIdentifier, int index, String nullFormat,
-      DictionaryClient client, Boolean useOnePass, Map<Object, Integer> localCache,
-      boolean isEmptyBadRecord, String parentTablePath,
-      boolean isConvertToBinary, String binaryDecoder)
-      throws IOException {
+  public FieldConverter createFieldEncoder(
+      DataField dataField, int index, String nullFormat, boolean isEmptyBadRecord,
+      boolean isConvertToBinary, String binaryDecoder) {
     // Converters are only needed for dimensions and measures it return null.
     if (dataField.getColumn().isDimension()) {
       if (dataField.getColumn().hasEncoding(Encoding.DIRECT_DICTIONARY) &&
           !dataField.getColumn().isComplex()) {
         return new DirectDictionaryFieldConverterImpl(dataField, nullFormat, index,
             isEmptyBadRecord);
-      } else if (dataField.getColumn().hasEncoding(Encoding.DICTIONARY) &&
-          !dataField.getColumn().isComplex()) {
-        DictionaryColumnUniqueIdentifier identifier = null;
-        // if parent column table relation is not null then it's a child table
-        // in case of child table it will use parent table dictionary
-        if (null == dataField.getColumn().getColumnSchema().getParentColumnTableRelations()
-            || dataField.getColumn().getColumnSchema().getParentColumnTableRelations().isEmpty()) {
-          identifier = new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-              dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType());
-          return new DictionaryFieldConverterImpl(dataField.getColumn(),
-              absoluteTableIdentifier.getCarbonTableIdentifier().getTableId(),
-              nullFormat, index, client, useOnePass, localCache, isEmptyBadRecord,
-              identifier);
-        } else {
-          ParentColumnTableRelation parentColumnTableRelation =
-              dataField.getColumn().getColumnSchema().getParentColumnTableRelations().get(0);
-          RelationIdentifier relationIdentifier =
-              parentColumnTableRelation
-                  .getRelationIdentifier();
-          CarbonTableIdentifier parentTableIdentifier =
-              new CarbonTableIdentifier(relationIdentifier.getDatabaseName(),
-                  relationIdentifier.getTableName(), relationIdentifier.getTableId());
-          ColumnIdentifier parentColumnIdentifier =
-              new ColumnIdentifier(parentColumnTableRelation.getColumnId(), null,
-                  dataField.getColumn().getDataType());
-          AbsoluteTableIdentifier parentAbsoluteTableIdentifier =
-              AbsoluteTableIdentifier.from(parentTablePath, parentTableIdentifier);
-          identifier = new DictionaryColumnUniqueIdentifier(parentAbsoluteTableIdentifier,
-              parentColumnIdentifier, dataField.getColumn().getDataType());
-          return new DictionaryFieldConverterImpl(dataField.getColumn(),
-              parentAbsoluteTableIdentifier.getCarbonTableIdentifier().getTableId(),
-              nullFormat, index, null, false, null, isEmptyBadRecord, identifier);
-        }
       } else if (dataField.getColumn().isComplex()) {
-        return new ComplexFieldConverterImpl(
-            createComplexDataType(dataField, absoluteTableIdentifier,
-                client, useOnePass, localCache, index, nullFormat, isEmptyBadRecord), index);
+        return new ComplexFieldConverterImpl(createComplexDataType(dataField, nullFormat), index);
       } else if (dataField.getColumn().getDataType() == DataTypes.BINARY) {
         BinaryDecoder binaryDecoderObject = null;
         if (binaryDecoder.equalsIgnoreCase(
@@ -191,12 +112,8 @@
   /**
    * Create parser for the carbon column.
    */
-  public static GenericDataType createComplexDataType(DataField dataField,
-      AbsoluteTableIdentifier absoluteTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      Map<Object, Integer> localCache, int index, String nullFormat, Boolean isEmptyBadRecords) {
-    return createComplexType(dataField.getColumn(), dataField.getColumn().getColName(),
-        absoluteTableIdentifier, client, useOnePass, localCache, index, nullFormat,
-        isEmptyBadRecords);
+  public static GenericDataType createComplexDataType(DataField dataField, String nullFormat) {
+    return createComplexType(dataField.getColumn(), dataField.getColumn().getColName(), nullFormat);
   }
 
   /**
@@ -206,8 +123,7 @@
    */
 
   private static GenericDataType createComplexType(CarbonColumn carbonColumn, String parentName,
-      AbsoluteTableIdentifier absoluteTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      Map<Object, Integer> localCache, int index, String nullFormat, Boolean isEmptyBadRecords) {
+      String nullFormat) {
     DataType dataType = carbonColumn.getDataType();
     if (DataTypes.isArrayType(dataType) || DataTypes.isMapType(dataType)) {
       List<CarbonDimension> listOfChildDimensions =
@@ -218,8 +134,7 @@
               carbonColumn.hasEncoding(Encoding.DICTIONARY));
       for (CarbonDimension dimension : listOfChildDimensions) {
         arrayDataType.addChildren(
-            createComplexType(dimension, carbonColumn.getColName(), absoluteTableIdentifier,
-                client, useOnePass, localCache, index, nullFormat, isEmptyBadRecords));
+            createComplexType(dimension, carbonColumn.getColName(), nullFormat));
       }
       return arrayDataType;
     } else if (DataTypes.isStructType(dataType)) {
@@ -231,14 +146,12 @@
               carbonColumn.hasEncoding(Encoding.DICTIONARY));
       for (CarbonDimension dimension : dimensions) {
         structDataType.addChildren(
-            createComplexType(dimension, carbonColumn.getColName(), absoluteTableIdentifier,
-                client, useOnePass, localCache, index, nullFormat, isEmptyBadRecords));
+            createComplexType(dimension, carbonColumn.getColName(), nullFormat));
       }
       return structDataType;
     } else {
       return new PrimitiveDataType(carbonColumn, parentName, carbonColumn.getColumnId(),
-          (CarbonDimension) carbonColumn, absoluteTableIdentifier, client, useOnePass,
-          localCache, nullFormat);
+          (CarbonDimension) carbonColumn, nullFormat);
     }
   }
 
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
index c1ed95d..eb6b5fa 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
@@ -20,20 +20,10 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.service.DictionaryOnePassService;
-import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.loading.BadRecordsLogger;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
@@ -66,12 +56,6 @@
 
   private BadRecordLogHolder logHolder;
 
-  private List<DictionaryClient> dictClients = new ArrayList<>();
-
-  private ExecutorService executorService;
-
-  private Map<Object, Integer>[] localCaches;
-
   private boolean isConvertToBinary;
 
   public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration,
@@ -98,70 +82,21 @@
         configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
             .toString());
     List<FieldConverter> fieldConverterList = new ArrayList<>();
-    localCaches = new Map[fields.length];
     long lruCacheStartTime = System.currentTimeMillis();
-    DictionaryClient client = createDictionaryClient();
-    dictClients.add(client);
 
     for (int i = 0; i < fields.length; i++) {
-      localCaches[i] = new ConcurrentHashMap<>();
       FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
-          .createFieldEncoder(fields[i], configuration.getTableIdentifier(), i, nullFormat, client,
-              configuration.getUseOnePass(), localCaches[i], isEmptyBadRecord,
-              configuration.getParentTablePath(), isConvertToBinary,
+          .createFieldEncoder(fields[i], i, nullFormat, isEmptyBadRecord, isConvertToBinary,
               (String) configuration.getDataLoadProperty(
                   CarbonLoadOptionConstants.CARBON_OPTIONS_BINARY_DECODER));
       fieldConverterList.add(fieldConverter);
     }
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
         .recordLruCacheLoadTime((System.currentTimeMillis() - lruCacheStartTime) / 1000.0);
-    fieldConverters = fieldConverterList.toArray(new FieldConverter[fieldConverterList.size()]);
+    fieldConverters = fieldConverterList.toArray(new FieldConverter[0]);
     logHolder = new BadRecordLogHolder();
   }
 
-  private DictionaryClient createDictionaryClient() {
-    // for one pass load, start the dictionary client
-    if (configuration.getUseOnePass()) {
-      if (executorService == null) {
-        executorService = Executors.newCachedThreadPool(new CarbonThreadFactory(
-            "DictionaryClientPool:" + configuration.getTableIdentifier().getCarbonTableIdentifier()
-                .getTableName(), true));
-      }
-      DictionaryOnePassService
-          .setDictionaryServiceProvider(configuration.getDictionaryServiceProvider());
-
-      Future<DictionaryClient> result =
-          executorService.submit(new Callable<DictionaryClient>() {
-            @Override
-            public DictionaryClient call() throws Exception {
-              Thread.currentThread().setName("Dictionary client");
-              DictionaryClient client =
-                  DictionaryOnePassService.getDictionaryProvider().getDictionaryClient();
-              client.startClient(configuration.getDictionaryServerSecretKey(),
-                  configuration.getDictionaryServerHost(), configuration.getDictionaryServerPort(),
-                  configuration.getDictionaryEncryptServerSecure());
-              return client;
-            }
-          });
-
-
-      try {
-        // wait for client initialization finished, or will raise null pointer exception
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        LOGGER.error(e.getMessage(), e);
-        throw new RuntimeException(e);
-      }
-
-      try {
-        return result.get();
-      } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    return null;
-  }
-
   @Override
   public CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException {
     logHolder.setLogged(false);
@@ -191,25 +126,9 @@
 
   @Override
   public void finish() {
-    // Clear up dictionary cache access count.
     for (int i = 0; i < fieldConverters.length; i++) {
       fieldConverters[i].clear();
     }
-    // close dictionary client when finish write
-    if (configuration.getUseOnePass()) {
-      for (DictionaryClient client : dictClients) {
-        if (client != null) {
-          client.shutDown();
-        }
-      }
-      if (null != logHolder) {
-        logHolder.finish();
-      }
-      if (executorService != null) {
-        executorService.shutdownNow();
-        executorService = null;
-      }
-    }
   }
 
   @Override
@@ -218,8 +137,6 @@
         new RowConverterImpl(this.fields, this.configuration, this.badRecordLogger,
             this.isConvertToBinary);
     List<FieldConverter> fieldConverterList = new ArrayList<>();
-    DictionaryClient client = createDictionaryClient();
-    dictClients.add(client);
     String nullFormat =
         configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
             .toString();
@@ -227,21 +144,13 @@
         configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
             .toString());
     for (int i = 0; i < fields.length; i++) {
-      FieldConverter fieldConverter = null;
-      try {
-        fieldConverter = FieldEncoderFactory.getInstance()
-            .createFieldEncoder(fields[i], configuration.getTableIdentifier(), i, nullFormat,
-                client, configuration.getUseOnePass(), localCaches[i], isEmptyBadRecord,
-                configuration.getParentTablePath(), isConvertToBinary,
-                (String) configuration.getDataLoadProperty(
-                    CarbonLoadOptionConstants.CARBON_OPTIONS_BINARY_DECODER));
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
+      FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
+          .createFieldEncoder(fields[i], i, nullFormat, isEmptyBadRecord, isConvertToBinary,
+              (String) configuration.getDataLoadProperty(
+                  CarbonLoadOptionConstants.CARBON_OPTIONS_BINARY_DECODER));
       fieldConverterList.add(fieldConverter);
     }
-    converter.fieldConverters =
-        fieldConverterList.toArray(new FieldConverter[fieldConverterList.size()]);
+    converter.fieldConverters = fieldConverterList.toArray(new FieldConverter[0]);
     converter.logHolder = new BadRecordLogHolder();
     return converter;
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.java
deleted file mode 100644
index aa16bd3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.dictionary;
-
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-
-/**
- * Dictionary implementation along with dictionary server client to get new dictionary values
- */
-public class DictionaryServerClientDictionary implements BiDictionary<Integer, Object> {
-
-  private Dictionary dictionary;
-
-  private DictionaryClient client;
-
-  private Map<Object, Integer> localCache;
-
-  private DictionaryMessage dictionaryMessage;
-
-  private int base;
-
-  public DictionaryServerClientDictionary(Dictionary dictionary, DictionaryClient client,
-      DictionaryMessage key, Map<Object, Integer> localCache) {
-    this.dictionary = dictionary;
-    this.client = client;
-    this.dictionaryMessage = key;
-    this.localCache = localCache;
-    this.base = (dictionary == null ? 0 : dictionary.getDictionaryChunks().getSize() - 1);
-  }
-
-  @Override
-  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
-    Integer key = getKey(value);
-    if (key == null) {
-      dictionaryMessage.setData(value.toString());
-      DictionaryMessage dictionaryValue = client.getDictionary(dictionaryMessage);
-      key = dictionaryValue.getDictionaryValue();
-      synchronized (localCache) {
-        localCache.put(value, key);
-      }
-      return key + base;
-    }
-    return key;
-  }
-
-  @Override
-  public Integer getKey(Object value) {
-    Integer key = -1;
-    if (dictionary != null) {
-      key = dictionary.getSurrogateKey(value.toString());
-    }
-    if (key == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
-      key = localCache.get(value);
-      if (key != null) {
-        return key + base;
-      }
-    }
-    return key;
-  }
-
-  @Override
-  public Object getValue(Integer key) {
-    throw new UnsupportedOperationException("Not supported here");
-  }
-
-  @Override
-  public int size() {
-    dictionaryMessage.setType(DictionaryMessageType.SIZE);
-    return client.getDictionary(dictionaryMessage).getDictionaryValue() + base;
-  }
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java
index b2d463c..16b8d20 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.processing.loading.dictionary;
 
 import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 
 /**
@@ -34,7 +33,7 @@
   }
 
   @Override
-  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
+  public Integer getOrGenerateKey(Object value) {
     Integer key = getKey(value);
     if (key == null) {
       throw new UnsupportedOperationException("trying to add new entry in DirectDictionary");
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java
deleted file mode 100644
index b7bd9b7..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.dictionary;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-
-public class PreCreatedDictionary implements BiDictionary<Integer, Object> {
-
-  private Dictionary dictionary;
-
-  public PreCreatedDictionary(Dictionary dictionary) {
-    this.dictionary = dictionary;
-  }
-
-  @Override
-  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
-    Integer key = getKey(value);
-    if (key == null) {
-      throw new UnsupportedOperationException("trying to add new entry in PreCreatedDictionary");
-    }
-    return key;
-  }
-
-  @Override
-  public Integer getKey(Object value) {
-    return dictionary.getSurrogateKey(value.toString());
-  }
-
-  @Override
-  public String getValue(Integer key) {
-    return dictionary.getDictionaryValueForKey(key);
-  }
-
-  @Override
-  public int size() {
-    return dictionary.getDictionaryChunks().getSize();
-  }
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 4264d83..81b393c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -20,14 +20,11 @@
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.dictionary.service.DictionaryServiceProvider;
 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.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
@@ -44,12 +41,8 @@
 
   private String factFilePath;
 
-  private String colDictFilePath;
-
   private CarbonDataLoadSchema carbonDataLoadSchema;
 
-  private boolean aggLoadRequest;
-
   private String tablePath;
 
   private String parentTablePath;
@@ -75,11 +68,6 @@
   private String blocksID;
 
   /**
-   * Map from carbon dimension to pre defined dict file path
-   */
-  private HashMap<CarbonDimension, String> predefDictMap;
-
-  /**
    * task id, each spark task has a unique id
    */
   private String taskNo;
@@ -92,8 +80,6 @@
    */
   private Segment segment;
 
-  private String allDictPath;
-
   /**
    * escape Char
    */
@@ -148,35 +134,6 @@
   private String skipEmptyLine;
 
   /**
-   * Use one pass to generate dictionary
-   */
-  private boolean useOnePass;
-
-  /**
-   * dictionary server host
-   */
-  private String dictionaryServerHost;
-
-  /**
-   * dictionary sever port
-   */
-  private int dictionaryServerPort;
-
-  /**
-   * dictionary server communication Secret Key.
-   */
-  private String dictionaryServerSecretKey;
-
-  /**
-   * dictionary service provider.
-   */
-  private DictionaryServiceProvider dictionaryServiceProvider;
-
-  /**
-   * Dictionary Secure or not.
-   */
-  private Boolean dictionaryEncryptServerSecure;
-  /**
    * Pre fetch data from csv reader
    */
   private boolean preFetch;
@@ -306,14 +263,6 @@
     return complexDelimiters;
   }
 
-  public String getAllDictPath() {
-    return allDictPath;
-  }
-
-  public void setAllDictPath(String allDictPath) {
-    this.allDictPath = allDictPath;
-  }
-
   public String getCsvHeader() {
     return csvHeader;
   }
@@ -330,18 +279,6 @@
     this.csvHeaderColumns = csvHeaderColumns;
   }
 
-  public void initPredefDictMap() {
-    predefDictMap = new HashMap<>();
-  }
-
-  public String getPredefDictFilePath(CarbonDimension dimension) {
-    return predefDictMap.get(dimension);
-  }
-
-  public void setPredefDictMap(CarbonDimension dimension, String predefDictFilePath) {
-    this.predefDictMap.put(dimension, predefDictFilePath);
-  }
-
   /**
    * @return carbon dataload schema
    */
@@ -398,30 +335,6 @@
     this.factFilePath = factFilePath;
   }
 
-  /**
-   * @return external column dictionary file path
-   */
-  public String getColDictFilePath() {
-    return colDictFilePath;
-  }
-
-  /**
-   * set external column dictionary file path
-   *
-   * @param colDictFilePath
-   */
-  public void setColDictFilePath(String colDictFilePath) {
-    this.colDictFilePath = colDictFilePath;
-  }
-
-  public DictionaryServiceProvider getDictionaryServiceProvider() {
-    return dictionaryServiceProvider;
-  }
-
-  public void setDictionaryServiceProvider(DictionaryServiceProvider dictionaryServiceProvider) {
-    this.dictionaryServiceProvider = dictionaryServiceProvider;
-  }
-
   public String getSortColumnsBoundsStr() {
     return sortColumnsBoundsStr;
   }
@@ -450,7 +363,6 @@
     copy.tableName = tableName;
     copy.factFilePath = factFilePath;
     copy.databaseName = databaseName;
-    copy.aggLoadRequest = aggLoadRequest;
     copy.loadMetadataDetails = loadMetadataDetails;
     copy.csvHeader = csvHeader;
     copy.csvHeaderColumns = csvHeaderColumns;
@@ -473,12 +385,6 @@
     copy.maxColumns = maxColumns;
     copy.tablePath = tablePath;
     copy.carbonTransactionalTable = carbonTransactionalTable;
-    copy.useOnePass = useOnePass;
-    copy.dictionaryServerHost = dictionaryServerHost;
-    copy.dictionaryServerPort = dictionaryServerPort;
-    copy.dictionaryServerSecretKey = dictionaryServerSecretKey;
-    copy.dictionaryServiceProvider = dictionaryServiceProvider;
-    copy.dictionaryEncryptServerSecure = dictionaryEncryptServerSecure;
     copy.preFetch = preFetch;
     copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
     copy.skipEmptyLine = skipEmptyLine;
@@ -511,7 +417,6 @@
     copyObj.tableName = tableName;
     copyObj.factFilePath = null;
     copyObj.databaseName = databaseName;
-    copyObj.aggLoadRequest = aggLoadRequest;
     copyObj.loadMetadataDetails = loadMetadataDetails;
     copyObj.carbonDataLoadSchema = carbonDataLoadSchema;
     copyObj.csvHeader = header;
@@ -534,12 +439,6 @@
     copyObj.maxColumns = maxColumns;
     copyObj.tablePath = tablePath;
     copyObj.carbonTransactionalTable = carbonTransactionalTable;
-    copyObj.useOnePass = useOnePass;
-    copyObj.dictionaryServerHost = dictionaryServerHost;
-    copyObj.dictionaryServerPort = dictionaryServerPort;
-    copyObj.dictionaryServerSecretKey = dictionaryServerSecretKey;
-    copyObj.dictionaryServiceProvider = dictionaryServiceProvider;
-    copyObj.dictionaryEncryptServerSecure = dictionaryEncryptServerSecure;
     copyObj.preFetch = preFetch;
     copyObj.isEmptyDataBadRecord = isEmptyDataBadRecord;
     copyObj.skipEmptyLine = skipEmptyLine;
@@ -787,46 +686,6 @@
     this.badRecordsAction = badRecordsAction;
   }
 
-  public boolean getUseOnePass() {
-    return useOnePass;
-  }
-
-  public void setUseOnePass(boolean useOnePass) {
-    this.useOnePass = useOnePass;
-  }
-
-  public int getDictionaryServerPort() {
-    return dictionaryServerPort;
-  }
-
-  public void setDictionaryServerPort(int dictionaryServerPort) {
-    this.dictionaryServerPort = dictionaryServerPort;
-  }
-
-  public String getDictionaryServerSecretKey() {
-    return dictionaryServerSecretKey;
-  }
-
-  public void setDictionaryServerSecretKey(String dictionaryServerSecretKey) {
-    this.dictionaryServerSecretKey = dictionaryServerSecretKey;
-  }
-
-  public Boolean getDictionaryEncryptServerSecure() {
-    return dictionaryEncryptServerSecure;
-  }
-
-  public void setDictionaryEncryptServerSecure(Boolean dictionaryEncryptServerSecure) {
-    this.dictionaryEncryptServerSecure = dictionaryEncryptServerSecure;
-  }
-
-  public String getDictionaryServerHost() {
-    return dictionaryServerHost;
-  }
-
-  public void setDictionaryServerHost(String dictionaryServerHost) {
-    this.dictionaryServerHost = dictionaryServerHost;
-  }
-
   public boolean isPreFetch() {
     return preFetch;
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index 9d31cb4..eb1ded5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -105,13 +105,6 @@
     }

     model.setDateFormat(dateFormat);

     model.setTimestampformat(timestampFormat);

-    model.setUseOnePass(Boolean.parseBoolean(Maps.getOrDefault(options, "onepass", "false")));

-    model.setDictionaryServerHost(Maps.getOrDefault(options, "dicthost", null));

-    try {

-      model.setDictionaryServerPort(Integer.parseInt(Maps.getOrDefault(options, "dictport", "-1")));

-    } catch (NumberFormatException e) {

-      throw new InvalidLoadOptionException(e.getMessage());

-    }

     validateAndSetColumnCompressor(model);

     validateAndSetBinaryDecoder(model);

     return model;

@@ -159,7 +152,6 @@
     // Need to fill dimension relation

     carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema);

     String sort_scope = optionsFinal.get("sort_scope");

-    String single_pass = optionsFinal.get("single_pass");

     String bad_records_logger_enable = optionsFinal.get("bad_records_logger_enable");

     String bad_records_action = optionsFinal.get("bad_records_action");

     String bad_record_path = optionsFinal.get("bad_record_path");

@@ -171,8 +163,6 @@
     String complex_delimiter_level2 = optionsFinal.get("complex_delimiter_level_2");

     String complex_delimiter_level3 = optionsFinal.get("complex_delimiter_level_3");

     String complex_delimiter_level4 = optionsFinal.get("complex_delimiter_level_4");

-    String all_dictionary_path = optionsFinal.get("all_dictionary_path");

-    String column_dict = optionsFinal.get("columndict");

     validateDateTimeFormat(timestampformat, "TimestampFormat");

     validateDateTimeFormat(dateFormat, "DateFormat");

 

@@ -255,7 +245,6 @@
 

     carbonLoadModel.setSortScope(sort_scope);

     carbonLoadModel.setGlobalSortPartitions(global_sort_partitions);

-    carbonLoadModel.setUseOnePass(Boolean.parseBoolean(single_pass));

 

     if (delimiter.equalsIgnoreCase(complex_delimiter_level1) ||

         complex_delimiter_level1.equalsIgnoreCase(complex_delimiter_level2) ||

@@ -268,11 +257,8 @@
       carbonLoadModel.setComplexDelimiter(complex_delimiter_level3);

       carbonLoadModel.setComplexDelimiter(complex_delimiter_level4);

     }

-    // set local dictionary path, and dictionary file extension

-    carbonLoadModel.setAllDictPath(all_dictionary_path);

     carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimiter));

     carbonLoadModel.setCsvHeader(fileHeader);

-    carbonLoadModel.setColDictFilePath(column_dict);

 

     List<String> ignoreColumns = new ArrayList<>();

     if (!isDataFrame) {

diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
index 0a69d2f..b0206bc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
@@ -26,7 +26,6 @@
 
 import org.apache.carbondata.common.Maps;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
@@ -52,14 +51,12 @@
   /**
    * Based on the input options, fill and return data loading options with default value
    */
-  public static Map<String, String> fillOptionWithDefaultValue(
-      Map<String, String> options) throws InvalidLoadOptionException {
+  public static Map<String, String> fillOptionWithDefaultValue(Map<String, String> options) {
     Map<String, String> optionsFinal = new HashMap<>();
     optionsFinal.put("delimiter", Maps.getOrDefault(options, "delimiter", ","));
     optionsFinal.put("quotechar", Maps.getOrDefault(options, "quotechar", "\""));
     optionsFinal.put("fileheader", Maps.getOrDefault(options, "fileheader", ""));
     optionsFinal.put("commentchar", Maps.getOrDefault(options, "commentchar", "#"));
-    optionsFinal.put("columndict", Maps.getOrDefault(options, "columndict", null));
 
     optionsFinal.put(
         "escapechar",
@@ -108,10 +105,6 @@
             CarbonProperties.getInstance().getProperty(
                 CarbonLoadOptionConstants.CARBON_OPTIONS_SKIP_EMPTY_LINE)));
 
-    optionsFinal.put(
-        "all_dictionary_path",
-        Maps.getOrDefault(options, "all_dictionary_path", ""));
-
     optionsFinal.put("complex_delimiter_level_1",
         Maps.getOrDefault(options, "complex_delimiter_level_1",
             ComplexDelimitersEnum.COMPLEX_DELIMITERS_LEVEL_1.value()));
@@ -152,31 +145,6 @@
                 null)));
 
     optionsFinal.put("maxcolumns", Maps.getOrDefault(options, "maxcolumns", null));
-
-    String useOnePass = Maps.getOrDefault(
-        options,
-        "single_pass",
-        CarbonProperties.getInstance().getProperty(
-            CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-            CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim().toLowerCase();
-
-    boolean singlePass;
-
-    if (useOnePass.equalsIgnoreCase("true")) {
-      singlePass = true;
-    } else {
-      // when single_pass = false  and if either alldictionarypath
-      // or columnDict is configured the do not allow load
-      if (StringUtils.isNotEmpty(optionsFinal.get("all_dictionary_path")) ||
-          StringUtils.isNotEmpty(optionsFinal.get("columndict"))) {
-        throw new InvalidLoadOptionException(
-            "Can not use all_dictionary_path or columndict without single_pass.");
-      } else {
-        singlePass = false;
-      }
-    }
-
-    optionsFinal.put("single_pass", String.valueOf(singlePass));
     optionsFinal.put("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT);
     optionsFinal.put("sort_column_bounds", Maps.getOrDefault(options, "sort_column_bounds", ""));
     optionsFinal.put(CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB,
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java
index 975d3e0..789d157 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepWithNoConverterImpl.java
@@ -105,20 +105,14 @@
 
   private void convertComplexDataType(Map<Integer, GenericDataType> dataFieldsWithComplexDataType) {
     DataField[] srcDataField = configuration.getDataFields();
-    FieldEncoderFactory fieldConverterFactory = FieldEncoderFactory.getInstance();
     String nullFormat =
         configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
             .toString();
-    boolean isEmptyBadRecord = Boolean.parseBoolean(
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
-            .toString());
     for (int i = 0; i < srcDataField.length; i++) {
       if (srcDataField[i].getColumn().isComplex()) {
         // create a ComplexDataType
         dataFieldsWithComplexDataType.put(srcDataField[i].getColumn().getOrdinal(),
-            fieldConverterFactory
-                .createComplexDataType(srcDataField[i], configuration.getTableIdentifier(), null,
-                    false, null, i, nullFormat, isEmptyBadRecord));
+            FieldEncoderFactory.createComplexDataType(srcDataField[i], nullFormat));
       }
     }
   }
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index 12bd2dd..3b8c2f5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -43,7 +43,6 @@
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthEquiSplitGenerator;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -51,7 +50,6 @@
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
 import org.apache.carbondata.processing.store.writer.CarbonFactDataWriter;
 
@@ -531,38 +529,7 @@
       LOGGER.debug("Number of rows per column page is configured as pageSize = " + pageSize);
     }
     dataRows = new ArrayList<>(this.pageSize);
-    int dimSet =
-        Integer.parseInt(CarbonCommonConstants.DIMENSION_SPLIT_VALUE_IN_COLUMNAR_DEFAULTVALUE);
-    // if at least one dimension is present then initialize column splitter otherwise null
-    int[] keyBlockSize = new int[getExpandedComplexColsCount()];
-
-    // agg type
-    List<Integer> otherMeasureIndexList =
-        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<Integer> customMeasureIndexList =
-        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    DataType[] type = model.getMeasureDataType();
-    for (int j = 0; j < type.length; j++) {
-      if (type[j] != DataTypes.BYTE && !DataTypes.isDecimal(type[j])) {
-        otherMeasureIndexList.add(j);
-      } else {
-        customMeasureIndexList.add(j);
-      }
-    }
-
-    int[] otherMeasureIndex = new int[otherMeasureIndexList.size()];
-    int[] customMeasureIndex = new int[customMeasureIndexList.size()];
-    for (int i = 0; i < otherMeasureIndex.length; i++) {
-      otherMeasureIndex[i] = otherMeasureIndexList.get(i);
-    }
-    for (int i = 0; i < customMeasureIndex.length; i++) {
-      customMeasureIndex[i] = customMeasureIndexList.get(i);
-    }
     setComplexMapSurrogateIndex(model.getDimensionCount());
-    int[] blockKeySize = getBlockKeySizeWithComplexTypes(new MultiDimKeyVarLengthEquiSplitGenerator(
-        CarbonUtil.getIncrementedCardinalityFullyFilled(model.getDimLens().clone()), (byte) dimSet)
-        .getBlockKeySize());
-    System.arraycopy(blockKeySize, 0, keyBlockSize, 0, blockKeySize.length);
     this.dataWriter = getFactDataWriter();
     // initialize the channel;
     this.dataWriter.initializeWriter();
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
index ebd21e8..c547d98 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/TablePage.java
@@ -108,8 +108,7 @@
       TableSpec.DimensionSpec spec = tableSpec.getDimensionSpec(i);
       ColumnType columnType = tableSpec.getDimensionSpec(i).getColumnType();
       ColumnPage page;
-      if (ColumnType.GLOBAL_DICTIONARY == columnType
-          || ColumnType.DIRECT_DICTIONARY == columnType) {
+      if (ColumnType.DIRECT_DICTIONARY == columnType) {
         page = ColumnPage.newPage(
             new ColumnPageEncoderMeta(spec, DataTypes.BYTE_ARRAY, columnCompressor), pageSize);
         page.setStatsCollector(KeyPageStatsCollector.newInstance(DataTypes.BYTE_ARRAY));
@@ -380,7 +379,6 @@
       EncodedColumnPage encodedPage;
       TableSpec.DimensionSpec spec = tableSpec.getDimensionSpec(i);
       switch (spec.getColumnType()) {
-        case GLOBAL_DICTIONARY:
         case DIRECT_DICTIONARY:
           columnPageEncoder = encodingFactory.createEncoder(
               spec,
@@ -434,7 +432,7 @@
     int numDimensions = spec.getNumDimensions();
     for (int i = 0; i < numDimensions; i++) {
       ColumnType type = spec.getDimensionSpec(i).getColumnType();
-      if ((type == ColumnType.GLOBAL_DICTIONARY) || (type == ColumnType.DIRECT_DICTIONARY)) {
+      if (type == ColumnType.DIRECT_DICTIONARY) {
         page = dictDimensionPages[++dictDimensionIndex];
       } else if (type == ColumnType.PLAIN_VALUE) {
         page = noDictDimensionPages[++noDictDimensionIndex];
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index f714dbf..a9acb24 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -47,9 +47,7 @@
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.locks.LockUsage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.SegmentFileStore;
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -545,13 +543,6 @@
     return dictCache.get(columnIdentifier);
   }
 
-  public static Dictionary getDictionary(AbsoluteTableIdentifier absoluteTableIdentifier,
-      ColumnIdentifier columnIdentifier, DataType dataType)
-      throws IOException {
-    return getDictionary(
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier, dataType));
-  }
-
   /**
    * This method will divide the blocks among the tasks of the nodes as per the data locality
    *