You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by xu...@apache.org on 2018/11/15 08:48:20 UTC

[1/2] carbondata git commit: [CARBONDATA-3087] Improve DESC FORMATTED output

Repository: carbondata
Updated Branches:
  refs/heads/master ceb135175 -> 851dd2c88


http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/test/scala/org/apache/spark/sql/GetDataSizeAndIndexSizeTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/sql/GetDataSizeAndIndexSizeTest.scala b/integration/spark2/src/test/scala/org/apache/spark/sql/GetDataSizeAndIndexSizeTest.scala
index 03ec3a1..563206f 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/sql/GetDataSizeAndIndexSizeTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/sql/GetDataSizeAndIndexSizeTest.scala
@@ -17,7 +17,10 @@
 
 package org.apache.spark.sql
 
+import java.util.Date
+
 import org.apache.spark.sql.test.util.QueryTest
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.scalatest.BeforeAndAfterAll
 
@@ -59,7 +62,7 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
       row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res1.length == 2)
-    res1.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res1.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
   }
 
   test("get data size and index size after major compaction") {
@@ -73,7 +76,7 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res2.length == 2)
-    res2.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res2.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
   }
 
   test("get data size and index size after minor compaction") {
@@ -91,7 +94,7 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res3.length == 2)
-    res3.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res3.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
   }
 
   test("get data size and index size after insert into") {
@@ -105,7 +108,7 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res4.length == 2)
-    res4.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res4.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
   }
 
   test("get data size and index size after insert overwrite") {
@@ -119,7 +122,7 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res5.length == 2)
-    res5.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res5.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
   }
 
   test("get data size and index size for empty table") {
@@ -128,15 +131,14 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res6.length == 2)
-    res6.foreach(row => assert(row.getString(1).trim.toLong == 0))
+    res6.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble == 0))
   }
 
   test("get last update time for empty table") {
     sql("CREATE TABLE tableSize9 (empno int, workgroupcategory string, deptno int, projectcode int, attendance int) STORED BY 'org.apache.carbondata.format'")
     val res7 = sql("DESCRIBE FORMATTED tableSize9").collect()
-      .filter(row => row.getString(0).contains(CarbonCommonConstants.LAST_UPDATE_TIME))
+      .filter(row => row.getString(0).contains("Last Update"))
     assert(res7.length == 1)
-    res7.foreach(row => assert(row.getString(1).trim.toLong == 0))
   }
 
   test("get last update time for unempty table") {
@@ -144,9 +146,8 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE tableSize10 OPTIONS ('DELIMITER'= ',', 'QUOTECHAR'= '\"', 'FILEHEADER'='')""")
 
     val res8 = sql("DESCRIBE FORMATTED tableSize10").collect()
-      .filter(row => row.getString(0).contains(CarbonCommonConstants.LAST_UPDATE_TIME))
+      .filter(row => row.getString(0).contains("Last Update"))
     assert(res8.length == 1)
-    res8.foreach(row => assert(row.getString(1).trim.toLong > 0))
   }
 
   test("index and datasize for update scenario") {
@@ -160,13 +161,13 @@ class GetDataSizeAndIndexSizeTest extends QueryTest with BeforeAndAfterAll {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
                      row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res9.length == 2)
-    res9.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res9.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
     sql("update tableSize11 set (empno) = (234)").show()
     val res10 = sql("DESCRIBE FORMATTED tableSize11").collect()
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
                      row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res10.length == 2)
-    res10.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res10.foreach(row => assert(row.getString(1).trim.substring(0, 2).toDouble > 0))
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
----------------------------------------------------------------------
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 bcca915..4926cd8 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
@@ -25,6 +25,7 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
+import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -37,7 +38,6 @@ import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 import org.apache.carbondata.processing.loading.steps.CarbonRowDataWriterProcessorStepImpl;
 import org.apache.carbondata.processing.loading.steps.DataConverterProcessorStepImpl;
 import org.apache.carbondata.processing.loading.steps.DataWriterBatchProcessorStepImpl;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
----------------------------------------------------------------------
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 03b2645..f69f0af 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
@@ -31,6 +31,7 @@ import org.apache.carbondata.common.constants.LoggerAction;
 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.SortScopeOptions;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -38,7 +39,6 @@ import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 import org.apache.carbondata.processing.util.CarbonBadRecordUtil;
 import org.apache.carbondata.processing.util.TableOptionConstant;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java
deleted file mode 100644
index 7b3c3df..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java
+++ /dev/null
@@ -1,49 +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.sort;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * Sort scope options
- */
-public class SortScopeOptions {
-
-  public static SortScope getSortScope(String sortScope) {
-    if (sortScope == null) {
-      sortScope = CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT;
-    }
-    switch (sortScope.toUpperCase()) {
-      case "BATCH_SORT":
-        return SortScope.BATCH_SORT;
-      case "LOCAL_SORT":
-        return SortScope.LOCAL_SORT;
-      case "GLOBAL_SORT":
-        return SortScope.GLOBAL_SORT;
-      case "NO_SORT":
-        return SortScope.NO_SORT;
-      default:
-        return SortScope.LOCAL_SORT;
-    }
-  }
-
-  public enum SortScope {
-    NO_SORT, BATCH_SORT, LOCAL_SORT, GLOBAL_SORT
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
index aad9083..af57759 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
@@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.sort.impl.ParallelReadMergeSorterImpl;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index 4012774..92c48bc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
@@ -48,7 +49,6 @@ import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
index 48993c5..f168796 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
@@ -45,7 +45,7 @@ import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.TABLE_BLOCKLET_SIZE;
 import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB;
 import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE;
-import static org.apache.carbondata.processing.loading.sort.SortScopeOptions.SortScope.NO_SORT;
+import static org.apache.carbondata.core.constants.SortScopeOptions.SortScope.NO_SORT;
 
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index b34ed13..437f628 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -33,6 +33,7 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.constants.LoggerAction;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -52,7 +53,6 @@ import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 import org.apache.carbondata.processing.loading.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;


[2/2] carbondata git commit: [CARBONDATA-3087] Improve DESC FORMATTED output

Posted by xu...@apache.org.
[CARBONDATA-3087] Improve DESC FORMATTED output

Change output of DESC FORMATTED

This closes #2908


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

Branch: refs/heads/master
Commit: 851dd2c884895762af305fa668ad80c151ba89bd
Parents: ceb1351
Author: Jacky Li <ja...@qq.com>
Authored: Thu Nov 8 20:11:28 2018 +0800
Committer: xuchuanyin <xu...@hust.edu.cn>
Committed: Thu Nov 15 16:48:04 2018 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  13 +-
 .../core/constants/SortScopeOptions.java        |  49 +++
 .../core/metadata/schema/table/CarbonTable.java |  75 ++++-
 .../core/metadata/schema/table/TableInfo.java   |   4 +-
 .../apache/carbondata/core/util/CarbonUtil.java |   3 +
 .../TestNoInvertedIndexLoadAndQuery.scala       |   7 +-
 .../preaggregate/TestPreAggCreateCommand.scala  |   7 +-
 ...ithColumnMetCacheAndCacheLevelProperty.scala |  12 +-
 ...ithColumnMetCacheAndCacheLevelProperty.scala |   8 +-
 .../TestCreateTableWithCompactionOptions.scala  |  20 --
 .../TestNonTransactionalCarbonTable.scala       |  52 +--
 .../testsuite/dataload/TestLoadDataFrame.scala  |   4 +-
 .../describeTable/TestDescribeTable.scala       |  12 +-
 .../LocalDictionarySupportCreateTableTest.scala |   8 +-
 .../testsuite/sortcolumns/TestSortColumns.scala |   3 +-
 .../sql/commands/StoredAsCarbondataSuite.scala  |   2 +-
 .../sql/commands/UsingCarbondataSuite.scala     |   2 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |   2 +-
 .../apache/spark/sql/test/util/QueryTest.scala  |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   3 +-
 .../management/CarbonLoadDataCommand.scala      |   3 +-
 .../table/CarbonDescribeFormattedCommand.scala  | 329 ++++++++++---------
 .../carbondata/TestStreamingTableOpName.scala   |   2 +-
 .../AlterTableValidationTestCase.scala          |   6 +-
 .../vectorreader/AddColumnTestCases.scala       |  38 +--
 .../vectorreader/ChangeDataTypeTestCases.scala  |   6 +-
 .../vectorreader/DropColumnTestCases.scala      |   6 +-
 .../spark/sql/GetDataSizeAndIndexSizeTest.scala |  25 +-
 .../loading/DataLoadProcessBuilder.java         |   2 +-
 .../loading/model/CarbonLoadModelBuilder.java   |   2 +-
 .../loading/sort/SortScopeOptions.java          |  49 ---
 .../processing/loading/sort/SorterFactory.java  |   1 +
 .../store/CarbonFactDataHandlerModel.java       |   2 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   2 +-
 .../util/CarbonDataProcessorUtil.java           |   2 +-
 35 files changed, 390 insertions(+), 373 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 259f84e..b75648e 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
@@ -62,11 +62,6 @@ public final class CarbonCommonConstants {
   public static final int BLOCKLET_SIZE_MAX_VAL = 12000000;
 
   /**
-   * default block size in MB
-   */
-  public static final String BLOCK_SIZE_DEFAULT_VAL = "1024";
-
-  /**
    * min block size in MB
    */
   public static final int BLOCK_SIZE_MIN_VAL = 1;
@@ -438,8 +433,16 @@ public final class CarbonCommonConstants {
   public static final String COLUMN_PROPERTIES = "columnproperties";
   // table block size in MB
   public static final String TABLE_BLOCKSIZE = "table_blocksize";
+
+  // default block size in MB
+  public static final String TABLE_BLOCK_SIZE_DEFAULT = "1024";
+
   // table blocklet size in MB
   public static final String TABLE_BLOCKLET_SIZE = "table_blocklet_size";
+
+  // default blocklet size value in MB
+  public static final String TABLE_BLOCKLET_SIZE_DEFAULT = "64";
+
   /**
    * set in column level to disable inverted index
    * @Deprecated :This property is deprecated, it is kept just for compatibility

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/core/src/main/java/org/apache/carbondata/core/constants/SortScopeOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/SortScopeOptions.java b/core/src/main/java/org/apache/carbondata/core/constants/SortScopeOptions.java
new file mode 100644
index 0000000..281a27e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/constants/SortScopeOptions.java
@@ -0,0 +1,49 @@
+/*
+ * 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.constants;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * Sort scope options
+ */
+public class SortScopeOptions {
+
+  public static SortScope getSortScope(String sortScope) {
+    if (sortScope == null) {
+      sortScope = CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT;
+    }
+    switch (sortScope.toUpperCase()) {
+      case "BATCH_SORT":
+        return SortScope.BATCH_SORT;
+      case "LOCAL_SORT":
+        return SortScope.LOCAL_SORT;
+      case "GLOBAL_SORT":
+        return SortScope.GLOBAL_SORT;
+      case "NO_SORT":
+        return SortScope.NO_SORT;
+      default:
+        return SortScope.LOCAL_SORT;
+    }
+  }
+
+  public enum SortScope {
+    NO_SORT, BATCH_SORT, LOCAL_SORT, GLOBAL_SORT
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 2f68754..82bf148 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
@@ -24,12 +24,15 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
+import org.apache.carbondata.core.constants.SortScopeOptions;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
@@ -55,6 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
 import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -62,6 +66,7 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 import static org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV;
 import static org.apache.carbondata.core.util.CarbonUtil.thriftColumnSchemaToWrapperColumnSchema;
 
+import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
 
@@ -841,6 +846,15 @@ public class CarbonTable implements Serializable {
     return blockSize;
   }
 
+  public int getBlockletSizeInMB() {
+    try {
+      return Integer.parseInt(tableInfo.getFactTable().getTableProperties().get(
+          CarbonCommonConstants.TABLE_BLOCKLET_SIZE));
+    } catch (NumberFormatException e) {
+      return Integer.parseInt(CarbonCommonConstants.TABLE_BLOCKLET_SIZE_DEFAULT);
+    }
+  }
+
   /**
    * to get the normal dimension or the primitive dimension of the complex type
    *
@@ -921,6 +935,10 @@ public class CarbonTable implements Serializable {
     return sort_columsList;
   }
 
+  public List<String> getSortColumns() {
+    return getSortColumns(getTableName());
+  }
+
   public int getNumberOfSortColumns() {
     return numberOfSortColumns;
   }
@@ -1214,16 +1232,22 @@ public class CarbonTable implements Serializable {
     String tableName = tableInfo.getFactTable().getTableName();
     String cacheColumns =
         tableInfo.getFactTable().getTableProperties().get(CarbonCommonConstants.COLUMN_META_CACHE);
-    if (null != cacheColumns && !cacheColumns.isEmpty()) {
-      String[] cachedCols = cacheColumns.split(",");
-      for (String column : cachedCols) {
-        CarbonColumn carbonColumn = getColumnByName(tableName, column);
-        if (null != carbonColumn && !carbonColumn.isInvisible()) {
-          cachedColsList.add(carbonColumn.getColName());
+    if (null != cacheColumns) {
+      if (!cacheColumns.isEmpty()) {
+        String[] cachedCols = cacheColumns.split(",");
+        for (String column : cachedCols) {
+          CarbonColumn carbonColumn = getColumnByName(tableName, column);
+          if (null != carbonColumn && !carbonColumn.isInvisible()) {
+            cachedColsList.add(carbonColumn.getColName());
+          }
         }
+        return cachedColsList;
+      } else {
+        return new LinkedList<>();
       }
+    } else {
+      return Lists.newArrayList("All columns");
     }
-    return cachedColsList;
   }
 
   /**
@@ -1297,4 +1321,41 @@ public class CarbonTable implements Serializable {
     }
     return false;
   }
+
+  /**
+   * Return all inverted index columns in this table
+   */
+  public List<ColumnSchema> getInvertedIndexColumns() {
+    if (getSortScope() == SortScopeOptions.SortScope.NO_SORT) {
+      return new LinkedList<>();
+    }
+    List<ColumnSchema> columns = new LinkedList<>();
+    for (ColumnSchema column : tableInfo.getFactTable().getListOfColumns()) {
+      if (column.isUseInvertedIndex() && column.isSortColumn()) {
+        columns.add(column);
+      }
+    }
+    return columns;
+  }
+
+  /**
+   * Return table level sort scope
+   */
+  public SortScopeOptions.SortScope getSortScope() {
+    String sortScope = tableInfo.getFactTable().getTableProperties().get("sort_scope");
+    if (sortScope == null) {
+      if (getNumberOfSortColumns() == 0) {
+        return SortScopeOptions.SortScope.NO_SORT;
+      } else {
+        return SortScopeOptions.getSortScope(
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
+                CarbonProperties.getInstance().getProperty(
+                    CarbonCommonConstants.LOAD_SORT_SCOPE,
+                    CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)));
+      }
+    } else {
+      return SortScopeOptions.getSortScope(sortScope);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 3e50586..daba29b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -257,12 +257,12 @@ public class TableInfo implements Serializable, Writable {
       tableBlockSize = tableProperties.get(CarbonCommonConstants.TABLE_BLOCKSIZE);
     }
     if (null == tableBlockSize) {
-      tableBlockSize = CarbonCommonConstants.BLOCK_SIZE_DEFAULT_VAL;
+      tableBlockSize = CarbonCommonConstants.TABLE_BLOCK_SIZE_DEFAULT;
       if (LOGGER.isDebugEnabled()) {
         LOGGER.debug(
             "Table block size not specified for " + getTableUniqueName() +
                 ". Therefore considering the default value " +
-                CarbonCommonConstants.BLOCK_SIZE_DEFAULT_VAL + " MB");
+                CarbonCommonConstants.TABLE_BLOCK_SIZE_DEFAULT + " MB");
       }
     }
     return Integer.parseInt(tableBlockSize);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 2fa6260..ac52728 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
@@ -2513,6 +2513,9 @@ public final class CarbonUtil {
                 FileFactory.getCarbonFile(tableStatusPath, FileFactory.getFileType(tableStatusPath))
                     .getLastModifiedTime();
           }
+          if (!FileFactory.isFileExist(metadataPath)) {
+            dataSize = FileFactory.getDirectorySize(carbonTable.getTablePath());
+          }
           dataIndexSizeMap
               .put(String.valueOf(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE), dataSize);
           dataIndexSizeMap

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
----------------------------------------------------------------------
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 0033556..13f8adb 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
@@ -283,7 +283,12 @@ class TestNoInvertedIndexLoadAndQuery extends QueryTest with BeforeAndAfterAll {
         """
            describe formatted indexFormat
         """),
-      true,"NOINVERTEDINDEX")
+      true,"Inverted Index Columns")
+
+    sql(
+      """
+           describe formatted indexFormat
+        """).show(100, false)
   }
 
   test("filter query on dictionary and no inverted index column where all values are null"){

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
index f0c335d..9fbdff7 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -77,7 +77,8 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
     sql("create datamap preagg11 on table PreAggMain1 using 'preaggregate'as select a,sum(b) from PreAggMain1 group by a")
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_a")
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_b_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "DICTIONARY")
+    sql("DESCRIBE FORMATTED PreAggMain1_preagg11").show(100, false)
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "Dictionary")
     sql("drop datamap preagg11 on table PreAggMain1")
   }
 
@@ -87,7 +88,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_b_sum")
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), false, "preaggmain1_a1")
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), false, "preaggmain1_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "DICTIONARY")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "Dictionary")
     sql("drop datamap preagg12 on table PreAggMain1")
   }
 
@@ -97,7 +98,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_b_sum")
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), false, "preaggmain1_a1")
     checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), false, "preaggmain1_sum")
-    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "DICTIONARY")
+    checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "Dictionary")
     sql("drop datamap preagg14 on table PreAggMain1")
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
index 8d4be3c..4c045c5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableWithColumnMetCacheAndCacheLevelProperty.scala
@@ -111,18 +111,18 @@ class TestAlterTableWithColumnMetCacheAndCacheLevelProperty extends QueryTest wi
   test("validate unsetting of column_meta_cache when column_meta_cache is already set - alter_column_meta_cache_11") {
     sql("Alter table alter_column_meta_cache SET TBLPROPERTIES('column_meta_cache'='c2,c3')")
     var descResult = sql("describe formatted alter_column_meta_cache")
-    checkExistence(descResult, true, "COLUMN_META_CACHE")
+    checkExistence(descResult, true, "Cached Min/Max Index Columns c2, c3")
     sql("Alter table alter_column_meta_cache UNSET TBLPROPERTIES('column_meta_cache')")
     descResult = sql("describe formatted alter_column_meta_cache")
-    checkExistence(descResult, false, "COLUMN_META_CACHE")
+    checkExistence(descResult, false, "Cached Min/Max Index Columns c2, c3")
   }
 
   test("validate unsetting of column_meta_cache when column_meta_cache is not already set - alter_column_meta_cache_12") {
     var descResult = sql("describe formatted alter_column_meta_cache")
-    checkExistence(descResult, false, "COLUMN_META_CACHE")
+    checkExistence(descResult, false, "c2, c3")
     sql("Alter table alter_column_meta_cache UNSET TBLPROPERTIES('column_meta_cache')")
     descResult = sql("describe formatted alter_column_meta_cache")
-    checkExistence(descResult, false, "COLUMN_META_CACHE")
+    checkExistence(descResult, false, "c2, c3")
   }
 
   test("validate cache_level with only empty spaces - ALTER_CACHE_LEVEL_01") {
@@ -150,14 +150,14 @@ class TestAlterTableWithColumnMetCacheAndCacheLevelProperty extends QueryTest wi
   test("validate describe formatted command to display cache_level when cache_level is set - ALTER_CACHE_LEVEL_05") {
     sql("Alter table cache_level SET TBLPROPERTIES('cache_level'='bloCKlet')")
     val descResult = sql("describe formatted cache_level")
-    checkExistence(descResult, true, "CACHE_LEVEL")
+    checkExistence(descResult, true, "Min/Max Index Cache Level")
   }
 
   test("validate describe formatted command to display cache_level when cache_level is not set - ALTER_CACHE_LEVEL_06") {
     sql("Alter table cache_level UNSET TBLPROPERTIES('cache_level')")
     val descResult = sql("describe formatted cache_level")
     // even though not configured default cache level will be displayed as BLOCK
-    checkExistence(descResult, true, "CACHE_LEVEL")
+    checkExistence(descResult, true, "Min/Max Index Cache Level")
   }
 
   test("validate column_meta_cache and cache_level on child dataMap- ALTER_CACHE_LEVEL_07") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithColumnMetCacheAndCacheLevelProperty.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithColumnMetCacheAndCacheLevelProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithColumnMetCacheAndCacheLevelProperty.scala
index 56aef40..4496405 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithColumnMetCacheAndCacheLevelProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithColumnMetCacheAndCacheLevelProperty.scala
@@ -109,14 +109,14 @@ class TestCreateTableWithColumnMetCacheAndCacheLevelProperty extends QueryTest w
     sql("drop table if exists column_meta_cache")
     sql("create table column_meta_cache(c1 String, c2 String, c3 int, c4 double) stored by 'carbondata' TBLPROPERTIES('COLUMN_meta_CachE'='c2')")
     val descResult = sql("describe formatted column_meta_cache")
-    checkExistence(descResult, true, "COLUMN_META_CACHE")
+    checkExistence(descResult, true, "Cached Min/Max Index Columns c2")
   }
 
   test("validate describe formatted command to display column_meta_cache when column_meta_cache is not set - COLUMN_META_CACHE_11") {
     sql("drop table if exists column_meta_cache")
     sql("create table column_meta_cache(c1 String, c2 String, c3 int, c4 double) stored by 'carbondata'")
     val descResult = sql("describe formatted column_meta_cache")
-    checkExistence(descResult, false, "COLUMN_META_CACHE")
+    checkExistence(descResult, false, "Cached Min/Max Index Columns c2")
   }
 
   test("validate column_meta_cache after column drop - COLUMN_META_CACHE_12") {
@@ -157,7 +157,7 @@ class TestCreateTableWithColumnMetCacheAndCacheLevelProperty extends QueryTest w
     sql("drop table if exists cache_level")
     sql("create table cache_level(c1 String) stored by 'carbondata' TBLPROPERTIES('cache_level'='bloCKlet')")
     val descResult = sql("describe formatted cache_level")
-    checkExistence(descResult, true, "CACHE_LEVEL")
+    checkExistence(descResult, true, "Min/Max Index Cache Level BLOCKLET")
   }
 
   test("validate describe formatted command to display cache_level when cache_level is not set - CACHE_LEVEL_06") {
@@ -165,7 +165,7 @@ class TestCreateTableWithColumnMetCacheAndCacheLevelProperty extends QueryTest w
     sql("create table cache_level(c1 String) stored by 'carbondata'")
     val descResult = sql("describe formatted cache_level")
     // even though not configured default cache level will be displayed as BLOCK
-    checkExistence(descResult, true, "CACHE_LEVEL")
+    checkExistence(descResult, true, "Min/Max Index Cache Level BLOCK")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithCompactionOptions.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithCompactionOptions.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithCompactionOptions.scala
index d7c05ce..5be774b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithCompactionOptions.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithCompactionOptions.scala
@@ -175,24 +175,4 @@ class TestCreateTableWithCompactionOptions extends QueryTest with BeforeAndAfter
         "only int value between 0 and 100 is supported."))
   }
 
-  test("test create table without compaction options") {
-    sql(
-      s"""
-         | CREATE TABLE $tableWithoutCompactionOptions(
-         | intField INT,
-         | stringField STRING
-         | )
-         | STORED BY 'carbondata'
-       """.stripMargin)
-
-    val tableOptions = sql(s"DESCRIBE FORMATTED $tableWithoutCompactionOptions")
-      .collect().map(r => (r.getString(0).trim, r.getString(1).trim)).toMap
-
-    assert(!tableOptions.contains("MAJOR_COMPACTION_SIZE"))
-    assert(!tableOptions.contains("AUTO_LOAD_MERGE"))
-    assert(!tableOptions.contains("COMPACTION_LEVEL_THRESHOLD"))
-    assert(!tableOptions.contains("COMPACTION_PRESERVE_SEGMENTS"))
-    assert(!tableOptions.contains("ALLOWED_COMPACTION_DAYS"))
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
----------------------------------------------------------------------
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 cce4ff0..942ad48 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
@@ -389,12 +389,6 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
          |'carbondata' LOCATION
          |'$writerPath' """.stripMargin)
 
-    val output = sql("Carboncli for table sdkOutputTable options('-cmd summary -a -v -c age')").collect()
-
-    assert(output.toList.contains(Row("written_by                       Version         ")))
-
-    checkExistence(sql("describe formatted sdkOutputTable"), true, "age,name")
-
     checkExistence(sql("describe formatted sdkOutputTable"), true, writerPath)
 
     buildTestDataWithSortColumns(List("age"))
@@ -406,9 +400,9 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
          |'carbondata' LOCATION
          |'$writerPath' """.stripMargin)
 
-    checkExistence(sql("describe formatted sdkOutputTable"), true, "SORT_COLUMNS                        age")
-    checkExistence(sql("describe formatted sdkOutputTable"), false, "SORT_COLUMNS                        name,age")
-    checkExistence(sql("describe formatted sdkOutputTable"), false, "SORT_COLUMNS                        age,name")
+    checkExistence(sql("describe formatted sdkOutputTable"), true, "Sort Columns age")
+    checkExistence(sql("describe formatted sdkOutputTable"), false, "Sort Columns name, age")
+    checkExistence(sql("describe formatted sdkOutputTable"), false, "Sort Columns age, name")
     buildTestDataSingleFile()
     assert(new File(writerPath).exists())
     sql("DROP TABLE IF EXISTS sdkOutputTable")
@@ -418,7 +412,7 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
          |'carbondata' LOCATION
          |'$writerPath' """.stripMargin)
 
-    checkExistence(sql("describe formatted sdkOutputTable"), true, "SORT_COLUMNS                        name")
+    checkExistence(sql("describe formatted sdkOutputTable"), true, "Sort Columns name")
 
     buildTestDataWithSortColumns(List())
     assert(new File(writerPath).exists())
@@ -430,8 +424,7 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
          |'carbondata' LOCATION
          |'$writerPath' """.stripMargin)
 
-    checkExistence(sql("describe formatted sdkOutputTable"),false,"SORT_COLUMNS                        name")
-    sql("select * from sdkOutputTable").show()
+    checkExistence(sql("describe formatted sdkOutputTable"),false,"Sort Columns name")
 
     sql("DROP TABLE sdkOutputTable")
     // drop table should not delete the files
@@ -882,7 +875,7 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
 
     // test the default sort column behavior in Nontransactional table
     checkExistence(sql("describe formatted sdkOutputTable"), true,
-      "SORT_COLUMNS                        name")
+      "Sort Columns name")
 
     sql("DROP TABLE sdkOutputTable")
     // drop table should not delete the files
@@ -2431,15 +2424,9 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""CREATE EXTERNAL TABLE sdkTable STORED BY 'carbondata' LOCATION
          |'$writerPath' """.stripMargin)
-    val descLoc = sql("describe formatted sdkTable").collect
-    descLoc.find(_.get(0).toString.contains("Local Dictionary Enabled")) match {
-      case Some(row) => assert(row.get(1).toString.contains("true"))
-      case None => assert(false)
-    }
-    descLoc.find(_.get(0).toString.contains("name")) match {
-      case Some(row) => assert(row.get(2).toString.contains("INVERTEDINDEX"))
-      case None => assert(false)
-    }
+    val df = sql("describe formatted sdkTable")
+    checkExistence(df, true, "Local Dictionary Enabled true")
+    checkExistence(df, true, "Inverted Index Columns name")
     FileUtils.deleteDirectory(new File(writerPath))
   }
 
@@ -2456,11 +2443,8 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""CREATE EXTERNAL TABLE sdkTable STORED BY 'carbondata' LOCATION
          |'$writerPath' """.stripMargin)
-    val descLoc = sql("describe formatted sdkTable").collect
-    descLoc.find(_.get(0).toString.contains("Local Dictionary Enabled")) match {
-      case Some(row) => assert(row.get(1).toString.contains("true"))
-      case None => assert(false)
-    }
+    val df = sql("describe formatted sdkTable")
+    checkExistence(df, true, "Local Dictionary Enabled true")
     FileUtils.deleteDirectory(new File(writerPath))
   }
 
@@ -2481,11 +2465,8 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
     sql("insert into sdkTable select 's1','s2',23 ")
     assert(FileFactory.getCarbonFile(writerPath).exists())
     assert(testUtil.checkForLocalDictionary(testUtil.getDimRawChunk(0,writerPath)))
-    val descLoc = sql("describe formatted sdkTable").collect
-    descLoc.find(_.get(0).toString.contains("Local Dictionary Enabled")) match {
-      case Some(row) => assert(row.get(1).toString.contains("true"))
-      case None => assert(false)
-    }
+    val df = sql("describe formatted sdkTable")
+    checkExistence(df, true, "Local Dictionary Enabled true")
     checkAnswer(sql("select count(*) from sdkTable"), Seq(Row(1)))
     FileUtils.deleteDirectory(new File(writerPath))
   }
@@ -2503,11 +2484,8 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
          |'$writerPath' """.stripMargin)
     FileUtils.deleteDirectory(new File(writerPath))
     sql("insert into sdkTable select 's1','s2',23 ")
-    val descLoc = sql("describe formatted sdkTable").collect
-    descLoc.find(_.get(0).toString.contains("name")) match {
-      case Some(row) => assert(row.get(2).toString.contains("INVERTEDINDEX"))
-      case None => assert(false)
-    }
+    val df = sql("describe formatted sdkTable")
+    checkExistence(df, true, "Inverted Index Columns name")
     checkAnswer(sql("select count(*) from sdkTable"), Seq(Row(1)))
     FileUtils.deleteDirectory(new File(writerPath))
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataFrame.scala
----------------------------------------------------------------------
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 f413b12..3186ccd 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
@@ -269,7 +269,7 @@ class TestLoadDataFrame extends QueryTest with BeforeAndAfterAll {
       sql("SELECT decimal FROM carbon11"),Seq(Row(BigDecimal.valueOf(10000.00)),Row(BigDecimal.valueOf(1234.44))))
     val descResult =sql("desc formatted carbon11")
     val isStreaming: String = descResult.collect().find(row=>row(0).asInstanceOf[String].trim.equalsIgnoreCase("streaming")).get.get(1).asInstanceOf[String]
-    assert(isStreaming.contains("true"))
+    assert(isStreaming.contains("sink"))
   }
 
   test("test datasource table with specified char") {
@@ -287,7 +287,7 @@ class TestLoadDataFrame extends QueryTest with BeforeAndAfterAll {
   private def getSortColumnValue(tableName: String): Array[String] = {
     val desc = sql(s"desc formatted $tableName")
     val sortColumnRow = desc.collect.find(r =>
-      r(0).asInstanceOf[String].trim.equalsIgnoreCase("SORT_COLUMNS")
+      r(0).asInstanceOf[String].trim.equalsIgnoreCase("Sort Columns")
     )
     assert(sortColumnRow.isDefined)
     sortColumnRow.get.get(1).asInstanceOf[String].split(",")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
index 764c612..0f7a4f6 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
@@ -44,17 +44,7 @@ class TestDescribeTable extends QueryTest with BeforeAndAfterAll {
   }
 
   test("test describe formatted table") {
-    checkExistence(sql("DESC FORMATTED Desc1"), true,
-      "Table Block Size")
-  }
-
-  test("test describe formatted table desc1") {
-
-    val resultCol = Seq("", "", "##Detailed Column property", "##Detailed Table Information", "ADAPTIVE", "CARBON Store Path", "Comment", "Database Name", "Last Update Time",
-    "SORT_COLUMNS", "SORT_SCOPE", "CACHE_LEVEL", "Streaming", "Table Block Size", "Local Dictionary Enabled","Local Dictionary Include", "Local Dictionary Threshold","Table Data Size", "Table Index Size", "Table Name", "dec2col1", "dec2col2", "dec2col3", "dec2col4")
-    val resultRow: Seq[Row] = resultCol map(propName => Row(f"$propName%-36s"))
-    checkAnswer(sql("desc formatted DESC1").select("col_name"), resultRow)
-    assert(sql("desc formatted desc1").count() == 24)
+    checkExistence(sql("DESC FORMATTED Desc1"), true, "Table Block Size")
   }
 
   test("test describe formatted for partition table") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/localdictionary/LocalDictionarySupportCreateTableTest.scala
----------------------------------------------------------------------
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 86c6cd6..e0ecc4d 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
@@ -1511,7 +1511,7 @@ class LocalDictionarySupportCreateTableTest extends QueryTest with BeforeAndAfte
       case Some(row) => assert(row.get(1).toString.contains("true"))
       case None => assert(false)
     }
-    descLoc.find(_.get(0).toString.contains("SORT_SCOPE")) match {
+    descLoc.find(_.get(0).toString.contains("Sort Scope")) match {
       case Some(row) => assert(row.get(1).toString.contains("global_sort"))
       case None => assert(false)
     }
@@ -1533,7 +1533,7 @@ class LocalDictionarySupportCreateTableTest extends QueryTest with BeforeAndAfte
       case Some(row) => assert(row.get(1).toString.contains("true"))
       case None => assert(false)
     }
-    descLoc.find(_.get(0).toString.contains("SORT_SCOPE")) match {
+    descLoc.find(_.get(0).toString.contains("Sort Scope")) match {
       case Some(row) => assert(row.get(1).toString.contains("batch_sort"))
       case None => assert(false)
     }
@@ -1554,7 +1554,7 @@ class LocalDictionarySupportCreateTableTest extends QueryTest with BeforeAndAfte
       case Some(row) => assert(row.get(1).toString.contains("true"))
       case None => assert(false)
     }
-    descLoc.find(_.get(0).toString.contains("SORT_SCOPE")) match {
+    descLoc.find(_.get(0).toString.contains("Sort Scope")) match {
       case Some(row) => assert(row.get(1).toString.contains("no_sort"))
       case None => assert(false)
     }
@@ -1575,7 +1575,7 @@ class LocalDictionarySupportCreateTableTest extends QueryTest with BeforeAndAfte
       case Some(row) => assert(row.get(1).toString.contains("true"))
       case None => assert(false)
     }
-    descLoc.find(_.get(0).toString.contains("SORT_SCOPE")) match {
+    descLoc.find(_.get(0).toString.contains("Sort Scope")) match {
       case Some(row) => assert(row.get(1).toString.contains("local_sort"))
       case None => assert(false)
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
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 8d4dba3..2fabeb3 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
@@ -353,8 +353,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
 
   test("describe formatted for sort_columns") {
     sql("CREATE TABLE sorttableDesc (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,empname')")
-    checkExistence(sql("describe formatted sorttableDesc"),true,"SORT_COLUMNS")
-    checkExistence(sql("describe formatted sorttableDesc"),true,"empno,empname")
+    checkExistence(sql("describe formatted sorttableDesc"),true,"Sort Columns empno, empname")
   }
 
   test("duplicate columns in sort_columns") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
index 6400ed1..72a45a2 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/StoredAsCarbondataSuite.scala
@@ -61,7 +61,7 @@ class StoredAsCarbondataSuite extends QueryTest with BeforeAndAfterEach {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res3.length == 2)
-    res3.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res3.foreach(row => assert(row.getString(1).trim.substring(0, 3).toDouble > 0))
   }
 
   test("CARBONDATA-2262: Don't Support the syntax of 'STORED AS 'carbondata''") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/UsingCarbondataSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/UsingCarbondataSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/UsingCarbondataSuite.scala
index 097c9d9..74e04b0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/UsingCarbondataSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/sql/commands/UsingCarbondataSuite.scala
@@ -70,7 +70,7 @@ class UsingCarbondataSuite extends QueryTest with BeforeAndAfterEach {
       .filter(row => row.getString(0).contains(CarbonCommonConstants.TABLE_DATA_SIZE) ||
         row.getString(0).contains(CarbonCommonConstants.TABLE_INDEX_SIZE))
     assert(res3.length == 2)
-    res3.foreach(row => assert(row.getString(1).trim.toLong > 0))
+    res3.foreach(row => assert(row.getString(1).trim.substring(0, 4).toDouble > 0))
   }
 
   test("CARBONDATA-2396 Support Create Table As Select with 'using carbondata'") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 0b6a2a9..3752eef 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -47,7 +47,7 @@ import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.CarbonQueryUtil
 import org.apache.carbondata.spark.DataLoadResult
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil}
 
 /**
  * This partition class use to split by Host

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
index 5c3ace3..5a26dd5 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/util/QueryTest.scala
@@ -53,7 +53,7 @@ class QueryTest extends PlanTest {
    * @param keywords keyword in string array
    */
   def checkExistence(df: DataFrame, exists: Boolean, keywords: String*) {
-    val outputs = df.collect().map(_.mkString).mkString
+    val outputs = df.collect().map(_.mkString(" ")).mkString(" ")
     for (key <- keywords) {
       if (exists) {
         assert(outputs.contains(key), s"Failed for $df ($key doesn't exist in result)")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
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 dc52517..7e2e7d9 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
@@ -46,7 +46,7 @@ import org.apache.spark.sql.util.{CarbonException, SparkSQLUtil}
 
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.constants.{CarbonCommonConstants, SortScopeOptions}
 import org.apache.carbondata.core.datamap.{DataMapStoreManager, Segment}
 import org.apache.carbondata.core.datamap.status.DataMapStatusManager
 import org.apache.carbondata.core.datastore.block.{Distributable, TableBlockInfo}
@@ -71,7 +71,6 @@ import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInput
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
 import org.apache.carbondata.processing.util.{Auditor, CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
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 bba8af7..af406bb 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
@@ -51,7 +51,7 @@ import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, FileUtils}
 import org.apache.carbondata.common.Strings
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.converter.SparkDataTypeConverterImpl
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
+import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants, SortScopeOptions}
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -74,7 +74,6 @@ import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataE
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.{CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
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 ebef152..151359e 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
@@ -17,9 +17,10 @@
 
 package org.apache.spark.sql.execution.command.table
 
+import java.util.Date
+
 import scala.collection.JavaConverters._
 
-import org.apache.commons.lang.StringUtils
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
@@ -27,12 +28,12 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.SparkPlan
 import org.apache.spark.sql.execution.command.MetadataCommand
 import org.apache.spark.sql.hive.CarbonRelation
-import org.codehaus.jackson.map.ObjectMapper
 
+import org.apache.carbondata.common.Strings
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 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.ColumnSchema
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonUtil
 
 private[sql] case class CarbonDescribeFormattedCommand(
@@ -46,99 +47,186 @@ private[sql] case class CarbonDescribeFormattedCommand(
     val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
       .lookupRelation(tblIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
     setAuditTable(relation.databaseName, relation.tableName)
-    val mapper = new ObjectMapper()
-    val colProps = StringBuilder.newBuilder
-    val dims = relation.metaData.dims.map(x => x.toLowerCase)
     var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
-      val fieldName = field.name.toLowerCase
       val colComment = field.getComment().getOrElse("null")
-      val comment = if (dims.contains(fieldName)) {
-        val dimension = relation.metaData.carbonTable.getDimensionByName(
-          relation.carbonTable.getTableName, fieldName)
-        if (null != dimension.getColumnProperties && !dimension.getColumnProperties.isEmpty) {
-          colProps.append(fieldName).append(".")
-            .append(mapper.writeValueAsString(dimension.getColumnProperties))
-            .append(",")
-        }
-        if (dimension.hasEncoding(Encoding.DICTIONARY) &&
-            !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          "DICTIONARY, KEY COLUMN" + (if (dimension.hasEncoding(Encoding.INVERTED_INDEX)) {
-            ",INVERTEDINDEX".concat(",").concat(colComment)
-          } else {
-            ",NOINVERTEDINDEX".concat(",").concat(colComment)
-          })
-        } else {
-          "KEY COLUMN" + (if (dimension.hasEncoding(Encoding.INVERTED_INDEX)) {
-            ",INVERTEDINDEX".concat(",").concat(colComment)
-          } else {
-            ",NOINVERTEDINDEX".concat(",").concat(colComment)
-          })
-        }
-      } else {
-        "MEASURE".concat(",").concat(colComment)
-      }
+      (field.name, field.dataType.simpleString, colComment)
+    }
 
-      (field.name, field.dataType.simpleString, comment)
+    val carbonTable = relation.carbonTable
+    val tblProps = carbonTable.getTableInfo.getFactTable.getTableProperties.asScala
+    val sortScope = if (carbonTable.getNumberOfSortColumns == 0) {
+      "NO_SORT"
+    } else {
+      tblProps.getOrElse("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)
     }
-    val colPropStr = if (colProps.toString().trim().length() > 0) {
-      // drops additional comma at end
-      colProps.toString().dropRight(1)
+    val streaming: String = if (carbonTable.isStreamingSink) {
+      "sink"
+    } else if (carbonTable.isStreamingSource) {
+      "source"
     } else {
-      colProps.toString()
+      "false"
     }
-    val carbonTable = relation.carbonTable
-    results ++= Seq(("", "", ""), ("##Detailed Table Information", "", ""))
-    results ++= Seq(("Database Name", relation.carbonTable.getDatabaseName, "")
+
+    val catalog = sparkSession.sessionState.catalog
+    val catalogTable = catalog.getTableMetadata(tblIdentifier)
+
+    //////////////////////////////////////////////////////////////////////////////
+    // Table Basic Information
+    //////////////////////////////////////////////////////////////////////////////
+    results ++= Seq(
+      ("", "", ""),
+      ("## Detailed Table Information", "", ""),
+      ("Database", catalogTable.database, ""),
+      ("Table", catalogTable.identifier.table, ""),
+      ("Owner", catalogTable.owner, ""),
+      ("Created", new Date(catalogTable.createTime).toString, ""),
+      ("Location ", carbonTable.getTablePath, ""),
+      ("External", carbonTable.isExternalTable.toString, ""),
+      ("Transactional", carbonTable.isTransactionalTable.toString, ""),
+      ("Streaming", streaming, ""),
+      ("Table Block Size ", carbonTable.getBlockSizeInMB + " MB", ""),
+      ("Table Blocklet Size ", carbonTable.getBlockletSizeInMB + " MB", ""),
+      ("Comment", tblProps.getOrElse(CarbonCommonConstants.TABLE_COMMENT, ""), ""),
+      ("Bad Record Path", tblProps.getOrElse("bad_record_path", ""), ""),
+      ("Min Input Per Node Per Load",
+        Strings.formatSize(
+          tblProps.getOrElse(CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB,
+            CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB_DEFAULT).toFloat), ""),
+
+      //////////////////////////////////////////////////////////////////////////////
+      //  Index Information
+      //////////////////////////////////////////////////////////////////////////////
+
+      ("", "", ""),
+      ("## Index Information", "", ""),
+      ("Sort Scope", sortScope, ""),
+      ("Sort Columns", relation.metaData.carbonTable.getSortColumns.asScala.mkString(", "), ""),
+      ("Inverted Index Columns", carbonTable.getInvertedIndexColumns.asScala
+        .map(_.getColumnName).mkString(", "), ""),
+      ("Cached Min/Max Index Columns",
+        carbonTable.getMinMaxCachedColumnsInCreateOrder.asScala.mkString(", "), ""),
+      ("Min/Max Index Cache Level",
+        tblProps.getOrElse(CarbonCommonConstants.CACHE_LEVEL,
+          CarbonCommonConstants.CACHE_LEVEL_DEFAULT_VALUE), "")
     )
-    results ++= Seq(("Table Name", relation.carbonTable.getTableName, ""))
-    results ++= Seq(("CARBON Store Path ", carbonTable.getTablePath, ""))
 
-    val tblProps = carbonTable.getTableInfo.getFactTable.getTableProperties
+    //////////////////////////////////////////////////////////////////////////////
+    //  Encoding Information
+    //////////////////////////////////////////////////////////////////////////////
 
-    // Carbon table support table comment
-    val tableComment = tblProps.asScala.getOrElse(CarbonCommonConstants.TABLE_COMMENT, "")
-    results ++= Seq(("Comment", tableComment, ""))
-    results ++= Seq(("Table Block Size ", carbonTable.getBlockSizeInMB + " MB", ""))
-    val dataIndexSize = CarbonUtil.calculateDataIndexSize(carbonTable, false)
-    if (!dataIndexSize.isEmpty) {
-      results ++= Seq((CarbonCommonConstants.TABLE_DATA_SIZE,
-        dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE).toString, ""))
-      results ++= Seq((CarbonCommonConstants.TABLE_INDEX_SIZE,
-        dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE).toString, ""))
-      results ++= Seq((CarbonCommonConstants.LAST_UPDATE_TIME,
-        dataIndexSize.get(CarbonCommonConstants.LAST_UPDATE_TIME).toString, ""))
+    results ++= Seq(
+      ("", "", ""),
+      ("## 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, ""), ""))
     }
 
-    results ++= Seq(("SORT_SCOPE", tblProps.asScala.getOrElse("sort_scope", CarbonCommonConstants
-      .LOAD_SORT_SCOPE_DEFAULT), tblProps.asScala.getOrElse("sort_scope", CarbonCommonConstants
-      .LOAD_SORT_SCOPE_DEFAULT)))
-    // add Cache Level property
-    results ++= Seq(("CACHE_LEVEL", tblProps.asScala.getOrElse(CarbonCommonConstants.CACHE_LEVEL,
-      CarbonCommonConstants.CACHE_LEVEL_DEFAULT_VALUE), ""))
-    val isStreaming = tblProps.asScala.getOrElse("streaming", "false")
-    results ++= Seq(("Streaming", isStreaming, ""))
+    //////////////////////////////////////////////////////////////////////////////
+    // Compaction Information
+    //////////////////////////////////////////////////////////////////////////////
 
-    // longstring related info
-    if (tblProps.containsKey(CarbonCommonConstants.LONG_STRING_COLUMNS)) {
-      results ++= Seq((CarbonCommonConstants.LONG_STRING_COLUMNS.toUpperCase,
-        tblProps.get(CarbonCommonConstants.LONG_STRING_COLUMNS), ""))
+    results ++= Seq(
+      ("", "", ""),
+      ("## Compaction Information", "", ""),
+      (CarbonCommonConstants.TABLE_MAJOR_COMPACTION_SIZE.toUpperCase,
+        tblProps.getOrElse(CarbonCommonConstants.TABLE_MAJOR_COMPACTION_SIZE,
+        CarbonCommonConstants.DEFAULT_CARBON_MAJOR_COMPACTION_SIZE), ""),
+      (CarbonCommonConstants.TABLE_AUTO_LOAD_MERGE.toUpperCase,
+        tblProps.getOrElse(CarbonCommonConstants.TABLE_AUTO_LOAD_MERGE,
+        CarbonCommonConstants.DEFAULT_ENABLE_AUTO_LOAD_MERGE), ""),
+      (CarbonCommonConstants.TABLE_COMPACTION_LEVEL_THRESHOLD.toUpperCase,
+        tblProps.getOrElse(CarbonCommonConstants.TABLE_COMPACTION_LEVEL_THRESHOLD,
+        CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD), ""),
+      (CarbonCommonConstants.TABLE_COMPACTION_PRESERVE_SEGMENTS.toUpperCase,
+        tblProps.getOrElse(CarbonCommonConstants.TABLE_COMPACTION_PRESERVE_SEGMENTS,
+        CarbonCommonConstants.DEFAULT_PRESERVE_LATEST_SEGMENTS_NUMBER), ""),
+      (CarbonCommonConstants.TABLE_ALLOWED_COMPACTION_DAYS.toUpperCase,
+        tblProps.getOrElse(CarbonCommonConstants.TABLE_ALLOWED_COMPACTION_DAYS,
+        CarbonCommonConstants.DEFAULT_DAYS_ALLOWED_TO_COMPACT), "")
+    )
+
+    //////////////////////////////////////////////////////////////////////////////
+    // Partition Information
+    //////////////////////////////////////////////////////////////////////////////
+    val partitionInfo = carbonTable.getPartitionInfo()
+    if (partitionInfo != null) {
+      results ++= Seq(
+        ("", "", ""),
+        ("## Partition Information", "", ""),
+        ("Partition Type", partitionInfo.getPartitionType.toString, ""),
+        ("Partition Columns",
+          partitionInfo.getColumnSchemaList.asScala.map {
+            col => s"${col.getColumnName}:${col.getDataType.getName}"}.mkString(", "), ""),
+        ("Number of Partitions", partitionInfo.getNumPartitions.toString, ""),
+        ("Partitions Ids", partitionInfo.getPartitionIds.asScala.mkString(","), "")
+      )
+      if (partitionInfo.getPartitionType == PartitionType.RANGE) {
+        results ++= Seq(("Range", partitionInfo.getRangeInfo.asScala.mkString(", "), ""))
+      } else if (partitionInfo.getPartitionType == PartitionType.LIST) {
+        results ++= Seq(("List", partitionInfo.getListInfo.asScala.mkString(", "), ""))
+      }
     }
+    if (partitionSpec.nonEmpty) {
+      val partitions = sparkSession.sessionState.catalog.getPartition(tblIdentifier, partitionSpec)
+      results ++=
+      Seq(("", "", ""),
+        ("## Partition Information", "", ""),
+        ("Partition Type", "Hive", ""),
+        ("Partition Value:", partitions.spec.values.mkString("[", ",", "]"), ""),
+        ("Database:", tblIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase), ""),
+        ("Table:", tblIdentifier.table, ""))
+      if (partitions.storage.locationUri.isDefined) {
+        results ++= Seq(("Location:", partitions.storage.locationUri.get.toString, ""))
+      }
+      results ++= Seq(("Partition Parameters:", partitions.parameters.mkString(", "), ""))
+    }
+
+    //////////////////////////////////////////////////////////////////////////////
+    // Dynamic Information
+    //////////////////////////////////////////////////////////////////////////////
 
-    // load min size info
-    if (tblProps.containsKey(CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB)) {
-      results ++= Seq(("Minimum input data size per node for data loading",
-        tblProps.get(CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB), ""))
+    val dataIndexSize = CarbonUtil.calculateDataIndexSize(carbonTable, false)
+    if (!dataIndexSize.isEmpty) {
+      if (carbonTable.isTransactionalTable) {
+        results ++= Seq(
+          ("", "", ""),
+          ("## Dynamic Information", "", ""),
+          ("Table Data Size", Strings.formatSize(
+            dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE).floatValue()), ""),
+          ("Table Index Size", Strings.formatSize(
+            dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE).floatValue()), ""),
+          ("Last Update",
+            new Date(dataIndexSize.get(CarbonCommonConstants.LAST_UPDATE_TIME)).toString, "")
+        )
+      } else {
+        results ++= Seq(
+          ("", "", ""),
+          ("## Dynamic Information", "", ""),
+          ("Table Total Size", Strings.formatSize(
+            dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE).floatValue()), "")
+        )
+      }
     }
 
-    var isLocalDictEnabled = tblProps.asScala
-      .get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE)
+    results.map{case (c1, c2, c3) => Row(c1, c2, c3)}
+  }
+
+  private def getLocalDictDesc(
+      carbonTable: CarbonTable,
+      tblProps: Map[String, String]): Seq[(String, String, String)] = {
+    val isLocalDictEnabled = tblProps.get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE)
+    var results = Seq[(String, String, String)]()
     if (isLocalDictEnabled.isDefined) {
       val localDictEnabled = isLocalDictEnabled.get.split(",") { 0 }
       results ++= Seq(("Local Dictionary Enabled", localDictEnabled, ""))
       // if local dictionary is enabled, then only show other properties of local dictionary
       if (localDictEnabled.toBoolean) {
-        var localDictThreshold = tblProps.asScala
-          .getOrElse(CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
+        var localDictThreshold = tblProps.getOrElse(
+            CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD,
             CarbonCommonConstants.LOCAL_DICTIONARY_THRESHOLD_DEFAULT)
         val localDictionaryThreshold = localDictThreshold.split(",")
         localDictThreshold = localDictionaryThreshold { 0 }
@@ -152,8 +240,7 @@ private[sql] case class CarbonDescribeFormattedCommand(
         }
         results ++=
         Seq(("Local Dictionary Include", getDictColumnString(builder.toString().split(",")), ""))
-        if (tblProps.asScala
-          .get(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE).isDefined) {
+        if (tblProps.get(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE).isDefined) {
           val columns = carbonTable.getTableInfo.getFactTable.getListOfColumns.asScala
           val builder = new StringBuilder
           columns.foreach { column =>
@@ -174,6 +261,7 @@ private[sql] case class CarbonDescribeFormattedCommand(
 
     /**
      * return the string which has all comma separated columns
+     *
      * @param localDictColumns
      * @return
      */
@@ -183,90 +271,7 @@ private[sql] case class CarbonDescribeFormattedCommand(
       dictColumns.toString().patch(dictColumns.toString().lastIndexOf(","), "", 1)
     }
 
-
-    // show table level compaction options
-    if (tblProps.containsKey(CarbonCommonConstants.TABLE_MAJOR_COMPACTION_SIZE)) {
-      results ++= Seq((CarbonCommonConstants.TABLE_MAJOR_COMPACTION_SIZE.toUpperCase
-        , tblProps.get(CarbonCommonConstants.TABLE_MAJOR_COMPACTION_SIZE),
-        CarbonCommonConstants.DEFAULT_CARBON_MAJOR_COMPACTION_SIZE))
-    }
-    if (tblProps.containsKey(CarbonCommonConstants.TABLE_AUTO_LOAD_MERGE)) {
-      results ++= Seq((CarbonCommonConstants.TABLE_AUTO_LOAD_MERGE.toUpperCase,
-        tblProps.get(CarbonCommonConstants.TABLE_AUTO_LOAD_MERGE),
-        CarbonCommonConstants.DEFAULT_ENABLE_AUTO_LOAD_MERGE))
-    }
-    if (tblProps.containsKey(CarbonCommonConstants.TABLE_COMPACTION_LEVEL_THRESHOLD)) {
-      results ++= Seq((CarbonCommonConstants.TABLE_COMPACTION_LEVEL_THRESHOLD.toUpperCase,
-        tblProps.get(CarbonCommonConstants.TABLE_COMPACTION_LEVEL_THRESHOLD),
-        CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD))
-    }
-    if (tblProps.containsKey(CarbonCommonConstants.TABLE_COMPACTION_PRESERVE_SEGMENTS)) {
-      results ++= Seq((CarbonCommonConstants.TABLE_COMPACTION_PRESERVE_SEGMENTS.toUpperCase,
-        tblProps.get(CarbonCommonConstants.TABLE_COMPACTION_PRESERVE_SEGMENTS),
-        CarbonCommonConstants.DEFAULT_PRESERVE_LATEST_SEGMENTS_NUMBER))
-    }
-    if (tblProps.containsKey(CarbonCommonConstants.TABLE_ALLOWED_COMPACTION_DAYS)) {
-      results ++= Seq((CarbonCommonConstants.TABLE_ALLOWED_COMPACTION_DAYS.toUpperCase,
-        tblProps.get(CarbonCommonConstants.TABLE_ALLOWED_COMPACTION_DAYS),
-        CarbonCommonConstants.DEFAULT_DAYS_ALLOWED_TO_COMPACT))
-    }
-    if (tblProps.containsKey(CarbonCommonConstants.FLAT_FOLDER)) {
-      results ++= Seq((CarbonCommonConstants.FLAT_FOLDER.toUpperCase,
-        tblProps.get(CarbonCommonConstants.FLAT_FOLDER),
-        CarbonCommonConstants.DEFAULT_FLAT_FOLDER))
-    }
-
-    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
-    if (colPropStr.length() > 0) {
-      results ++= Seq((colPropStr, "", ""))
-    } else {
-      results ++= Seq(("ADAPTIVE", "", ""))
-    }
-    results ++= Seq(("SORT_COLUMNS", relation.metaData.carbonTable.getSortColumns(
-      relation.carbonTable.getTableName).asScala
-      .map(column => column).mkString(","), ""))
-
-    val bad_record_path = relation.carbonTable.getTableInfo.getFactTable
-      .getTableProperties.get("bad_record_path")
-    if (!StringUtils.isEmpty(bad_record_path)) {
-      results ++= Seq(("BAD_RECORD_PATH", bad_record_path, ""))
-    }
-    // add columns configured in column meta cache
-    if (null != tblProps.get(CarbonCommonConstants.COLUMN_META_CACHE)) {
-      results ++=
-      Seq(("COLUMN_META_CACHE", carbonTable.getMinMaxCachedColumnsInCreateOrder().asScala
-        .map(col => col).mkString(","), ""))
-    }
-    if (carbonTable.getPartitionInfo(carbonTable.getTableName) != null) {
-      results ++=
-      Seq(("#Partition Information", "", ""),
-        ("#col_name", "data_type", "comment"))
-      results ++= carbonTable.getPartitionInfo(carbonTable.getTableName)
-        .getColumnSchemaList.asScala.map {
-        col => (col.getColumnName, col.getDataType.getName, "NULL")
-      }
-      results ++= Seq(("Partition Type", carbonTable.getPartitionInfo(carbonTable.getTableName)
-        .getPartitionType.toString, ""))
-    }
-    if (partitionSpec.nonEmpty) {
-      val partitions = sparkSession.sessionState.catalog.getPartition(tblIdentifier, partitionSpec)
-      results ++=
-      Seq(("", "", ""),
-        ("##Detailed Partition Information", "", ""),
-        ("Partition Value:", partitions.spec.values.mkString("[", ",", "]"), ""),
-        ("Database:", tblIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase), ""),
-        ("Table:", tblIdentifier.table, ""))
-      if (partitions.storage.locationUri.isDefined) {
-        results ++= Seq(("Location:", partitions.storage.locationUri.get.toString, ""))
-      }
-      results ++= Seq(("Partition Parameters:", partitions.parameters.mkString(", "), ""))
-    }
-    results.map {
-      case (name, dataType, null) =>
-        Row(f"$name%-36s", f"$dataType%-80s", null)
-      case (name, dataType, comment) =>
-        Row(f"$name%-36s", f"$dataType%-80s", f"$comment%-72s")
-    }
+    results
   }
 
   override protected def opName: String = "DESC FORMATTED"

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOpName.scala
----------------------------------------------------------------------
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 d789f5c..f5596f2 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
@@ -1661,7 +1661,7 @@ class TestStreamingTableOpName extends QueryTest with BeforeAndAfterAll {
     val resultStreaming = sql("DESC FORMATTED stream_table_file").collect()
       .filter(_.getString(0).trim.equals("Streaming"))
     assertResult(1)(resultStreaming.length)
-    assertResult("true")(resultStreaming(0).getString(1).trim)
+    assertResult("sink")(resultStreaming(0).getString(1).trim)
   }
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
----------------------------------------------------------------------
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 054a996..dce2953 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
@@ -528,16 +528,14 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
         "tblproperties('DICTIONARY_INCLUDE'='empno','DICTIONARY_EXCLUDE'='role')")
     sql("alter table defaultSortColumnsWithAlter drop columns (designation)")
     sql("alter table defaultSortColumnsWithAlter add columns (designation12 String)")
-    checkExistence(sql("describe formatted defaultSortColumnsWithAlter"),true,"SORT_COLUMNS")
-    checkExistence(sql("describe formatted defaultSortColumnsWithAlter"),true,"empno,empname,role,doj")
+    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')")
     sql("alter table specifiedSortColumnsWithAlter drop columns (designation)")
     sql("alter table specifiedSortColumnsWithAlter add columns (designation12 String)")
-    checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"SORT_COLUMNS")
-    checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"empno,empname,role,doj")
+    checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"Sort Columns empno, empname, role, doj")
   }
 
   test("test to check if new parent table name is reflected in pre-aggregate tables") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
----------------------------------------------------------------------
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 625fb89..e0aa012 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
@@ -131,18 +131,19 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
   test("test compaction after adding new column") {
     sqlContext.setConf("carbon.enable.vector.reader", "true")
     sql("ALTER TABLE addcolumntest COMPACT 'major'")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "1Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0.1Success")
+    sql("SHOW SEGMENTS FOR TABLE addcolumntest").show(100, false)
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "1 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0.1 Success")
     checkAnswer(sql("SELECT charField FROM addcolumntest"), Seq(Row("abc"), Row("def")))
 
     afterAll
     beforeAll
     sqlContext.setConf("carbon.enable.vector.reader", "false")
     sql("ALTER TABLE addcolumntest COMPACT 'major'")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "1Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0.1Success")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "1 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE addcolumntest"), true, "0.1 Success")
     checkAnswer(sql("SELECT charField FROM addcolumntest"), Seq(Row("abc"), Row("def")))
   }
 
@@ -563,9 +564,9 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
     sql("ALTER TABLE alter_dict DROP COLUMNS(charField)")
     sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data7.csv' INTO TABLE alter_dict OPTIONS('FILEHEADER'='stringField')")
     sql("ALTER TABLE alter_dict COMPACT 'major'")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_dict"), true, "0Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_dict"), true, "1Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_dict"), true, "0.1Success")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_dict"), true, "0 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_dict"), true, "1 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_dict"), true, "0.1 Success")
     sql("DROP TABLE IF EXISTS alter_dict")
   }
 
@@ -590,9 +591,9 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
     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')")
     sql("ALTER TABLE alter_no_dict COMPACT 'major'")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_no_dict"), true, "0Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_no_dict"), true, "1Compacted")
-    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_no_dict"), true, "0.1Success")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_no_dict"), true, "0 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_no_dict"), true, "1 Compacted")
+    checkExistence(sql("SHOW SEGMENTS FOR TABLE alter_no_dict"), true, "0.1 Success")
     sql("DROP TABLE IF EXISTS alter_no_dict")
   }
 
@@ -641,7 +642,7 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
       """)
 
     sql("alter table NO_INVERTED_CARBON add columns(col1 string,col2 string) tblproperties('NO_INVERTED_INDEX'='col2')")
-    checkExistenceCount(sql("desc formatted NO_INVERTED_CARBON"),4,"NOINVERTEDINDEX")
+    checkExistence(sql("desc formatted NO_INVERTED_CARBON"),false,"Inverted Index Columns name, col1")
   }
 
   test("inverted index after alter command") {
@@ -655,15 +656,8 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
       """)
 
     sql("alter table NO_INVERTED_CARBON add columns(col1 string,col2 string) tblproperties('INVERTED_INDEX'='col2')")
-    val descLoc = sql("describe formatted NO_INVERTED_CARBON").collect
-    descLoc.find(_.get(0).toString.contains("name")) match {
-      case Some(row) => assert(row.get(2).toString.contains("INVERTEDINDEX"))
-      case None => assert(false)
-    }
-    descLoc.find(_.get(0).toString.contains("col2")) match {
-      case Some(row) => assert(row.get(2).toString.contains("INVERTEDINDEX"))
-      case None => assert(false)
-    }
+    val df = sql("describe formatted NO_INVERTED_CARBON")
+    checkExistence(df, true, "Inverted Index Columns city, col2")
   }
 
   test("test if adding column in pre-aggregate table throws exception") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
----------------------------------------------------------------------
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 04740de..4bf7de0 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
@@ -119,9 +119,9 @@ class ChangeDataTypeTestCases extends Spark2QueryTest with BeforeAndAfterAll {
       checkAnswer(sql("SELECT decimalField FROM changedatatypetest"),
         Seq(Row(new BigDecimal("21.23").setScale(5)), Row(new BigDecimal("21.23").setScale(5))))
       sql("ALTER TABLE changedatatypetest COMPACT 'major'")
-      checkExistence(sql("SHOW SEGMENTS FOR TABLE changedatatypetest"), true, "0Compacted")
-      checkExistence(sql("SHOW SEGMENTS FOR TABLE changedatatypetest"), true, "1Compacted")
-      checkExistence(sql("SHOW SEGMENTS FOR TABLE changedatatypetest"), true, "0.1Success")
+      checkExistence(sql("SHOW SEGMENTS FOR TABLE changedatatypetest"), true, "0 Compacted")
+      checkExistence(sql("SHOW SEGMENTS FOR TABLE changedatatypetest"), true, "1 Compacted")
+      checkExistence(sql("SHOW SEGMENTS FOR TABLE changedatatypetest"), true, "0.1 Success")
       afterAll
     }
     sqlContext.setConf("carbon.enable.vector.reader", "true")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/851dd2c8/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
index a713c7b..e6285dd 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
@@ -85,9 +85,9 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
       sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data4.csv' INTO TABLE dropcolumntest"
           + s" OPTIONS('FILEHEADER'='intField,stringField,timestampField,decimalField')")
       sql("ALTER TABLE dropcolumntest COMPACT 'major'")
-      checkExistence(sql("SHOW SEGMENTS FOR TABLE dropcolumntest"), true, "0Compacted")
-      checkExistence(sql("SHOW SEGMENTS FOR TABLE dropcolumntest"), true, "1Compacted")
-      checkExistence(sql("SHOW SEGMENTS FOR TABLE dropcolumntest"), true, "0.1Success")
+      checkExistence(sql("SHOW SEGMENTS FOR TABLE dropcolumntest"), true, "0 Compacted")
+      checkExistence(sql("SHOW SEGMENTS FOR TABLE dropcolumntest"), true, "1 Compacted")
+      checkExistence(sql("SHOW SEGMENTS FOR TABLE dropcolumntest"), true, "0.1 Success")
       afterAll
     }
     sqlContext.setConf("carbon.enable.vector.reader", "true")