You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/10/09 15:50:05 UTC

[03/45] carbondata git commit: [CARBONDATA-2965] support Benchmark command in CarbonCli

[CARBONDATA-2965] support Benchmark command in CarbonCli

A new command called "benchmark" is added in CarbonCli tool to output the scan performance of the specified file and column.
Example usage:
```bash
shell>java -jar carbondata-cli.jar org.apache.carbondata.CarbonCli -cmd benchmark -p hdfs://carbon1:9000/carbon.store/tpchcarbon_base/lineitem/ -a -c l_comment
```
will scan output the scan time of l_comment column in first file in the input folder and prints: (or using -f option to provide the data file instead of folder)

```
ReadHeaderAndFooter takes 12,598 us
ConvertFooter takes 4,712 us
ReadAllMetaAndConvertFooter takes 8,039 us

Scan column 'l_comment'
Blocklet#0: ColumnChunkIO takes 222,609 us
Blocklet#0: DecompressPage takes 111,985 us
Blocklet#1: ColumnChunkIO takes 186,522 us
Blocklet#1: DecompressPage takes 89,132 us
Blocklet#2: ColumnChunkIO takes 209,129 us
Blocklet#2: DecompressPage takes 84,051 us
```
This closes #2755


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

Branch: refs/heads/branch-1.5
Commit: e07df44a1db52304c54ab4e379f28b0f026449fd
Parents: 49f6715
Author: Jacky Li <ja...@qq.com>
Authored: Sun Sep 23 00:01:04 2018 +0800
Committer: xuchuanyin <xu...@hust.edu.cn>
Committed: Wed Sep 26 15:47:37 2018 +0800

----------------------------------------------------------------------
 .../core/util/DataFileFooterConverterV3.java    |   6 +-
 pom.xml                                         |   7 +-
 tools/cli/pom.xml                               |   5 +
 .../org/apache/carbondata/tool/CarbonCli.java   |  90 ++++----
 .../org/apache/carbondata/tool/Command.java     |  28 +++
 .../org/apache/carbondata/tool/DataFile.java    |  94 +++++++--
 .../org/apache/carbondata/tool/DataSummary.java | 188 ++++++-----------
 .../apache/carbondata/tool/FileCollector.java   | 147 +++++++++++++
 .../apache/carbondata/tool/ScanBenchmark.java   | 205 +++++++++++++++++++
 .../apache/carbondata/tool/CarbonCliTest.java   |  94 +++++----
 10 files changed, 622 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
index 41e22fd..438e3e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverterV3.java
@@ -59,12 +59,16 @@ public class DataFileFooterConverterV3 extends AbstractDataFileFooterConverter {
    */
   @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
       throws IOException {
-    DataFileFooter dataFileFooter = new DataFileFooter();
     CarbonHeaderReader carbonHeaderReader = new CarbonHeaderReader(tableBlockInfo.getFilePath());
     FileHeader fileHeader = carbonHeaderReader.readHeader();
     CarbonFooterReaderV3 reader =
         new CarbonFooterReaderV3(tableBlockInfo.getFilePath(), tableBlockInfo.getBlockOffset());
     FileFooter3 footer = reader.readFooterVersion3();
+    return convertDataFileFooter(fileHeader, footer);
+  }
+
+  public DataFileFooter convertDataFileFooter(FileHeader fileHeader, FileFooter3 footer) {
+    DataFileFooter dataFileFooter = new DataFileFooter();
     dataFileFooter.setVersionId(ColumnarFormatVersion.valueOf((short) fileHeader.getVersion()));
     dataFileFooter.setNumberOfRows(footer.getNum_rows());
     dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index eff438b..00a5287 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,7 @@
     <module>store/sdk</module>
     <module>store/search</module>
     <module>assembly</module>
+    <module>tools/cli</module>
   </modules>
 
   <properties>
@@ -718,12 +719,6 @@
         <module>datamap/mv/core</module>
       </modules>
     </profile>
-    <profile>
-      <id>tools</id>
-      <modules>
-        <module>tools/cli</module>
-      </modules>
-    </profile>
   </profiles>
 
 </project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/pom.xml
----------------------------------------------------------------------
diff --git a/tools/cli/pom.xml b/tools/cli/pom.xml
index 0d00438..60e69dc 100644
--- a/tools/cli/pom.xml
+++ b/tools/cli/pom.xml
@@ -25,6 +25,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>servlet-api</artifactId>
+      <version>2.5</version>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
index effb139..5725f8e 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
@@ -47,11 +47,16 @@ public class CarbonCli {
         .withDescription("the path which contains carbondata files, nested folder is supported")
         .withLongOpt("path")
         .create("p");
+    Option file = OptionBuilder.withArgName("file")
+        .hasArg()
+        .withDescription("the carbondata file path")
+        .withLongOpt("file")
+        .create("f");
 
     Option command = OptionBuilder
         .withArgName("command name")
         .hasArg()
-        .withDescription("command to execute, supported commands are: summary")
+        .withDescription("command to execute, supported commands are: summary, benchmark")
         .isRequired(true)
         .create("cmd");
 
@@ -70,6 +75,7 @@ public class CarbonCli {
     Options options = new Options();
     options.addOption(help);
     options.addOption(path);
+    options.addOption(file);
     options.addOption(command);
     options.addOption(all);
     options.addOption(schema);
@@ -87,71 +93,49 @@ public class CarbonCli {
   static void run(String[] args, PrintStream out) {
     Options options = buildOptions();
     CommandLineParser parser = new PosixParser();
-    try {
-      CommandLine line = parser.parse(options, args);
-      if (line.hasOption("h")) {
-        printHelp(options);
-        return;
-      }
-
-      String cmd = line.getOptionValue("cmd");
-      if (cmd.equalsIgnoreCase("summary")) {
-        runSummaryCommand(line, options, out);
-      } else {
-        out.println("command " + cmd + " is not supported");
-        printHelp(options);
-        return;
-      }
 
-      out.flush();
+    CommandLine line;
+    try {
+      line = parser.parse(options, args);
     } catch (ParseException exp) {
       out.println("Parsing failed. Reason: " + exp.getMessage());
-    } catch (IOException | MemoryException e) {
-      out.println(out);
+      return;
     }
-  }
 
-  private static void printHelp(Options options) {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("CarbonCli", options);
-  }
+    if (line.hasOption("h")) {
+      printHelp(options);
+      return;
+    }
 
-  private static void runSummaryCommand(CommandLine line, Options options, PrintStream out)
-      throws IOException, MemoryException {
     String path = "";
     if (line.hasOption("p")) {
       path = line.getOptionValue("path");
+    }
+    out.println("Input Folder: " + path);
+
+    String cmd = line.getOptionValue("cmd");
+    Command command;
+    if (cmd.equalsIgnoreCase("summary")) {
+      command = new DataSummary(path, out);
+    } else if (cmd.equalsIgnoreCase("benchmark")) {
+      command = new ScanBenchmark(path, out);
     } else {
-      System.err.println("path is required");
+      out.println("command " + cmd + " is not supported");
       printHelp(options);
       return;
     }
-    DataSummary summary = new DataSummary(path, out);
-    if (summary.isEmpty()) {
-      System.out.println("no data file found");
-      return;
-    }
-    out.println("Input Folder: " + path);
-    summary.printBasic();
-    boolean printAll = false;
-    if (line.hasOption("a")) {
-      printAll = true;
-    }
-    if (line.hasOption("s") || printAll) {
-      summary.printSchema();
-    }
-    if (line.hasOption("m") || printAll) {
-      summary.printSegments();
-    }
-    if (line.hasOption("t") || printAll) {
-      summary.printTableProperties();
-    }
-    if (line.hasOption("b") || printAll) {
-      summary.printBlockletDetail();
-    }
-    if (line.hasOption("c")) {
-      String columName = line.getOptionValue("c");
-      summary.printColumnStats(columName);
+
+    try {
+      command.run(line);
+      out.flush();
+    } catch (IOException | MemoryException e) {
+      e.printStackTrace();
     }
   }
+
+  private static void printHelp(Options options) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("CarbonCli", options);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java b/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java
new file mode 100644
index 0000000..cb7d8df
--- /dev/null
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/Command.java
@@ -0,0 +1,28 @@
+/*
+ * 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.tool;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.memory.MemoryException;
+
+import org.apache.commons.cli.CommandLine;
+
+interface Command {
+  void run(CommandLine line) throws IOException, MemoryException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java b/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
index ea67829..da81d84 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/DataFile.java
@@ -55,6 +55,8 @@ import static org.apache.carbondata.core.constants.CarbonCommonConstants.FILE_SE
  * Contains information extracted from a .carbondata file
  */
 class DataFile {
+  private CarbonFile dataFile;
+
   // file full path
   private String filePath;
 
@@ -80,33 +82,37 @@ class DataFile {
 
   private FileHeader header;
   private FileFooter3 footer;
+  private long footerOffset;
   private List<ColumnSchema> schema;
   private List<Blocklet> blocklets;
 
-  DataFile(CarbonFile file) throws IOException {
-    this.fileSizeInBytes = file.getSize();
+  DataFile(CarbonFile dataFile) {
+    this.dataFile = dataFile;
+    this.filePath = dataFile.getPath();
+    this.fileSizeInBytes = dataFile.getSize();
+  }
 
+  void collectAllMeta() throws IOException {
     FileHeader header = null;
     FileFooter3 footer = null;
     try {
-      header = readHeader(file);
+      header = readHeader();
     } catch (IOException e) {
-      throw new IOException("failed to read header in " + file.getPath(), e);
+      throw new IOException("failed to read header in " + dataFile.getPath(), e);
     }
     if (header.isSetSync_marker()) {
       // if sync_marker is set, it is a streaming format file
       throw new UnsupportedOperationException("streaming file is not supported");
     }
     try {
-      footer = readFooter(file);
+      footer = readFooter();
     } catch (IOException e) {
-      throw new IOException("failed to read footer in " + file.getPath(), e);
+      throw new IOException("failed to read footer in " + dataFile.getPath(), e);
     }
 
-    this.filePath = file.getPath();
     this.header = header;
     this.footer = footer;
-    String filePath = file.getPath();
+    String filePath = dataFile.getPath();
     // folder path that contains this file
     String fileName = filePath.substring(filePath.lastIndexOf(FILE_SEPARATOR));
     this.shardName = CarbonTablePath.getShardName(fileName);
@@ -130,6 +136,7 @@ class DataFile {
       for (int i = 0; i < schema.size(); i++) {
         columnDataSize.add(blockletInfo3.column_data_chunks_offsets.get(i) - previousChunkOffset);
         columnMetaSize.add(blockletInfo3.column_data_chunks_length.get(i).longValue());
+        previousChunkOffset = blockletInfo3.column_data_chunks_offsets.get(i);
       }
       // last column chunk data size
       columnDataSize.add(fileSizeInBytes - footerSizeInBytes - previousChunkOffset);
@@ -146,17 +153,17 @@ class DataFile {
     assert (blockletSizeInBytes.size() == getNumBlocklets());
   }
 
-  private FileHeader readHeader(CarbonFile dataFile) throws IOException {
+  FileHeader readHeader() throws IOException {
     CarbonHeaderReader reader = new CarbonHeaderReader(dataFile.getPath());
     this.schema = reader.readSchema();
     return reader.readHeader();
   }
 
-  private FileFooter3 readFooter(CarbonFile dataFile) throws IOException {
+  FileFooter3 readFooter() throws IOException {
     this.fileReader = FileFactory.getFileHolder(FileFactory.getFileType(dataFile.getPath()));
     ByteBuffer buffer = fileReader.readByteBuffer(FileFactory.getUpdatedFilePath(
         dataFile.getPath()), dataFile.getSize() - 8, 8);
-    long footerOffset = buffer.getLong();
+    this.footerOffset = buffer.getLong();
     this.footerSizeInBytes = this.fileSizeInBytes - footerOffset;
     CarbonFooterReaderV3 footerReader =
         new CarbonFooterReaderV3(dataFile.getAbsolutePath(), footerOffset);
@@ -187,6 +194,53 @@ class DataFile {
     return schema;
   }
 
+  FileReader getFileReader() {
+    return fileReader;
+  }
+
+  long getFooterOffset() {
+    return footerOffset;
+  }
+
+  int getNumBlocklet() {
+    return blockletSizeInBytes.size();
+  }
+
+  long getFileSizeInBytes() {
+    return fileSizeInBytes;
+  }
+
+  int getColumnIndex(String columnName) {
+    List<ColumnSchema> columns = getSchema();
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).getColumnName().equalsIgnoreCase(columnName)) {
+        return i;
+      }
+    }
+    throw new IllegalArgumentException(columnName + " not found");
+  }
+
+  ColumnSchema getColumn(String columnName) {
+    List<ColumnSchema> columns = getSchema();
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).getColumnName().equalsIgnoreCase(columnName)) {
+        return columns.get(i);
+      }
+    }
+    throw new IllegalArgumentException(columnName + " not found");
+  }
+
+  int numDimensions() {
+    int numDimensions = 0;
+    List<ColumnSchema> columns = getSchema();
+    for (ColumnSchema column : columns) {
+      if (column.isDimensionColumn()) {
+        numDimensions++;
+      }
+    }
+    return numDimensions;
+  }
+
   private int getNumBlocklets() {
     return footer.blocklet_info_list3.size();
   }
@@ -396,28 +450,34 @@ class DataFile {
         if (column.isSortColumn()) {
           minValue = ByteUtil.toXorInt(min, 0, min.length);
           dataValue = ByteUtil.toXorInt(data, 0, data.length) - minValue;
-          factorValue = ByteUtil.toXorInt(max, 0, max.length) - ByteUtil.toXorInt(min, 0, min.length);
+          factorValue =
+              ByteUtil.toXorInt(max, 0, max.length) - ByteUtil.toXorInt(min, 0, min.length);
         } else {
           minValue = ByteUtil.toLong(min, 0, min.length);
           dataValue = ByteUtil.toLong(data, 0, data.length) - minValue;
-          factorValue = ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
+          factorValue =
+              ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
         }
       } else if (column.getDataType() == DataTypes.LONG) {
         minValue = ByteUtil.toLong(min, 0, min.length);
         dataValue = ByteUtil.toLong(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
+        factorValue =
+            ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
       } else if (column.getDataType() == DataTypes.DATE) {
         minValue = ByteUtil.toInt(min, 0, min.length);
         dataValue = ByteUtil.toInt(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toInt(max, 0, max.length) - ByteUtil.toInt(min, 0, min.length);
+        factorValue =
+            ByteUtil.toInt(max, 0, max.length) - ByteUtil.toInt(min, 0, min.length);
       } else if (column.getDataType() == DataTypes.TIMESTAMP) {
         minValue = ByteUtil.toLong(min, 0, min.length);
         dataValue = ByteUtil.toLong(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
+        factorValue =
+            ByteUtil.toLong(max, 0, max.length) - ByteUtil.toLong(min, 0, min.length);
       } else if (column.getDataType() == DataTypes.DOUBLE) {
         minValue = ByteUtil.toDouble(min, 0, min.length);
         dataValue = ByteUtil.toDouble(data, 0, data.length) - minValue;
-        factorValue = ByteUtil.toDouble(max, 0, max.length) - ByteUtil.toDouble(min, 0, min.length);
+        factorValue =
+            ByteUtil.toDouble(max, 0, max.length) - ByteUtil.toDouble(min, 0, min.length);
       } else {
         throw new UnsupportedOperationException("data type: " + column.getDataType());
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java b/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
index 7ca6951..6463977 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/DataSummary.java
@@ -20,15 +20,12 @@ package org.apache.carbondata.tool;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.charset.Charset;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.carbondata.common.Strings;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
@@ -40,7 +37,6 @@ import org.apache.carbondata.core.reader.CarbonHeaderReader;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockletInfo3;
 import org.apache.carbondata.format.FileFooter3;
 import org.apache.carbondata.format.FileHeader;
@@ -48,135 +44,89 @@ import org.apache.carbondata.format.TableInfo;
 
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.DEFAULT_CHARSET;
 
+import org.apache.commons.cli.CommandLine;
+
 /**
  * Data Summary command implementation for {@link CarbonCli}
  */
-class DataSummary {
+class DataSummary implements Command {
   private String dataFolder;
   private PrintStream out;
 
-  private long numBlock;
-  private long numShard;
-  private long numBlocklet;
-  private long numPage;
-  private long numRow;
-  private long totalDataSize;
-
   // file path mapping to file object
-  private LinkedHashMap<String, DataFile> dataFiles = new LinkedHashMap<>();
-  private CarbonFile tableStatusFile;
-  private CarbonFile schemaFile;
+  private LinkedHashMap<String, DataFile> dataFiles;
 
-  DataSummary(String dataFolder, PrintStream out) throws IOException {
+  DataSummary(String dataFolder, PrintStream out) {
     this.dataFolder = dataFolder;
     this.out = out;
-    collectDataFiles();
-  }
-
-  private boolean isColumnarFile(String fileName) {
-    // if the timestamp in file name is "0", it is a streaming file
-    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
-        !CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
   }
 
-  private boolean isStreamFile(String fileName) {
-    // if the timestamp in file name is "0", it is a streaming file
-    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
-        CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
-  }
-
-  private void collectDataFiles() throws IOException {
-    Set<String> shards = new HashSet<>();
-    CarbonFile folder = FileFactory.getCarbonFile(dataFolder);
-    List<CarbonFile> files = folder.listFiles(true);
-    List<DataFile> unsortedFiles = new ArrayList<>();
-    for (CarbonFile file : files) {
-      if (isColumnarFile(file.getName())) {
-        DataFile dataFile = new DataFile(file);
-        unsortedFiles.add(dataFile);
-        collectNum(dataFile.getFooter());
-        shards.add(dataFile.getShardName());
-        totalDataSize += file.getSize();
-      } else if (file.getName().endsWith(CarbonTablePath.TABLE_STATUS_FILE)) {
-        tableStatusFile = file;
-      } else if (file.getName().startsWith(CarbonTablePath.SCHEMA_FILE)) {
-        schemaFile = file;
-      } else if (isStreamFile(file.getName())) {
-        out.println("WARN: input path contains streaming file, this tool does not support it yet, "
-            + "skipping it...");
-      }
+  @Override
+  public void run(CommandLine line) throws IOException, MemoryException {
+    FileCollector collector = new FileCollector(out);
+    collector.collectFiles(dataFolder);
+    collector.printBasicStats();
+    if (collector.getNumDataFiles() == 0) {
+      return;
     }
-    unsortedFiles.sort((o1, o2) -> {
-      if (o1.getShardName().equalsIgnoreCase(o2.getShardName())) {
-        return Integer.parseInt(o1.getPartNo()) - Integer.parseInt(o2.getPartNo());
-      } else {
-        return o1.getShardName().compareTo(o2.getShardName());
+    dataFiles = collector.getDataFiles();
+    boolean printAll = false;
+    if (line.hasOption("a")) {
+      printAll = true;
+    }
+    if (line.hasOption("s") || printAll) {
+      if (dataFiles.size() > 0) {
+        printSchema(dataFiles.entrySet().iterator().next().getValue());
       }
-    });
-    for (DataFile collectedFile : unsortedFiles) {
-      this.dataFiles.put(collectedFile.getFilePath(), collectedFile);
     }
-    numShard = shards.size();
-  }
-
-  private void collectNum(FileFooter3 footer) {
-    numBlock++;
-    numBlocklet += footer.blocklet_index_list.size();
-    numRow += footer.num_rows;
-    for (BlockletInfo3 blockletInfo3 : footer.blocklet_info_list3) {
-      numPage += blockletInfo3.number_number_of_pages;
+    if (line.hasOption("m") || printAll) {
+      printSegments(collector.getTableStatusFile());
+    }
+    if (line.hasOption("t") || printAll) {
+      printTableProperties(collector.getSchemaFile());
+    }
+    if (line.hasOption("b") || printAll) {
+      printBlockletDetail();
+    }
+    if (line.hasOption("c")) {
+      String columName = line.getOptionValue("c");
+      printColumnStats(columName);
     }
   }
 
-  void printBasic() {
-    out.println("## Summary");
-    out.println(
-        String.format("total: %,d blocks, %,d shards, %,d blocklets, %,d pages, %,d rows, %s",
-            numBlock, numShard, numBlocklet, numPage, numRow, Strings.formatSize(totalDataSize)));
-    out.println(
-        String.format("avg: %s/block, %s/blocklet, %,d rows/block, %,d rows/blocklet",
-            Strings.formatSize(totalDataSize / numBlock),
-            Strings.formatSize(totalDataSize / numBlocklet),
-            numRow / numBlock,
-            numRow / numBlocklet));
-  }
-
-  void printSchema() throws IOException {
-    if (dataFiles.size() > 0) {
-      String firstFile = dataFiles.keySet().iterator().next();
-      CarbonFile file = FileFactory.getCarbonFile(firstFile);
-      out.println();
-      out.println("## Schema");
-      out.println(String.format("schema in %s", file.getName()));
-      CarbonHeaderReader reader = new CarbonHeaderReader(file.getPath());
-      FileHeader header = reader.readHeader();
-      out.println("version: V" + header.version);
-      out.println("timestamp: " + new java.sql.Timestamp(header.time_stamp));
-      List<ColumnSchema> columns = reader.readSchema();
-      TablePrinter printer = new TablePrinter(
-          new String[]{"Column Name", "Data Type", "Column Type",
-              "SortColumn", "Encoding", "Ordinal", "Id"});
-      for (ColumnSchema column : columns) {
-        String shortColumnId = "NA";
-        if (column.getColumnUniqueId() != null && column.getColumnUniqueId().length() > 4) {
-          shortColumnId = "*" +
-              column.getColumnUniqueId().substring(column.getColumnUniqueId().length() - 4);
-        }
-        printer.addRow(new String[]{
-            column.getColumnName(),
-            column.getDataType().getName(),
-            column.isDimensionColumn() ? "dimension" : "measure",
-            String.valueOf(column.isSortColumn()),
-            column.getEncodingList().toString(),
-            Integer.toString(column.getSchemaOrdinal()),
-            shortColumnId
-        });
+  private void printSchema(DataFile dataFile) throws IOException {
+    CarbonFile file = FileFactory.getCarbonFile(dataFile.getFilePath());
+    out.println();
+    out.println("## Schema");
+    out.println(String.format("schema in %s", file.getName()));
+    CarbonHeaderReader reader = new CarbonHeaderReader(file.getPath());
+    FileHeader header = reader.readHeader();
+    out.println("version: V" + header.version);
+    out.println("timestamp: " + new java.sql.Timestamp(header.time_stamp));
+    List<ColumnSchema> columns = reader.readSchema();
+    TablePrinter printer = new TablePrinter(
+        new String[]{"Column Name", "Data Type", "Column Type",
+            "SortColumn", "Encoding", "Ordinal", "Id"});
+    for (ColumnSchema column : columns) {
+      String shortColumnId = "NA";
+      if (column.getColumnUniqueId() != null && column.getColumnUniqueId().length() > 4) {
+        shortColumnId = "*" +
+            column.getColumnUniqueId().substring(column.getColumnUniqueId().length() - 4);
       }
-      printer.printFormatted(out);
+      printer.addRow(new String[]{
+          column.getColumnName(),
+          column.getDataType().getName(),
+          column.isDimensionColumn() ? "dimension" : "measure",
+          String.valueOf(column.isSortColumn()),
+          column.getEncodingList().toString(),
+          Integer.toString(column.getSchemaOrdinal()),
+          shortColumnId
+      });
     }
+    printer.printFormatted(out);
   }
 
-  void printSegments() throws IOException {
+  private void printSegments(CarbonFile tableStatusFile) throws IOException {
     out.println();
     out.println("## Segment");
     if (tableStatusFile != null) {
@@ -215,7 +165,7 @@ class DataSummary {
     }
   }
 
-  void printTableProperties() throws IOException {
+  private void printTableProperties(CarbonFile schemaFile) throws IOException {
     out.println();
     out.println("## Table Properties");
     if (schemaFile != null) {
@@ -235,7 +185,7 @@ class DataSummary {
     }
   }
 
-  void printBlockletDetail() {
+  private void printBlockletDetail() {
     out.println();
     out.println("## Block Detail");
 
@@ -262,17 +212,12 @@ class DataSummary {
 
   private int getColumnIndex(String columnName) {
     if (dataFiles.size() > 0) {
-      List<ColumnSchema> columns = dataFiles.entrySet().iterator().next().getValue().getSchema();
-      for (int i = 0; i < columns.size(); i++) {
-        if (columns.get(i).getColumnName().equalsIgnoreCase(columnName)) {
-          return i;
-        }
-      }
+      return dataFiles.entrySet().iterator().next().getValue().getColumnIndex(columnName);
     }
     throw new RuntimeException("schema for column " + columnName + " not found");
   }
 
-  void printColumnStats(String columnName) throws IOException, MemoryException {
+  private void printColumnStats(String columnName) throws IOException, MemoryException {
     out.println();
     out.println("## Column Statistics for '" + columnName + "'");
     for (DataFile dataFile : dataFiles.values()) {
@@ -354,7 +299,4 @@ class DataSummary {
     }
   }
 
-  public boolean isEmpty() {
-    return dataFiles.size() == 0;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java b/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java
new file mode 100644
index 0000000..6e3297f
--- /dev/null
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/FileCollector.java
@@ -0,0 +1,147 @@
+/*
+ * 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.tool;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.BlockletInfo3;
+import org.apache.carbondata.format.FileFooter3;
+
+/**
+ * A helper to collect all data files, schema file, table status file in a given folder
+ */
+class FileCollector {
+
+  private long numBlock;
+  private long numShard;
+  private long numBlocklet;
+  private long numPage;
+  private long numRow;
+  private long totalDataSize;
+
+  // file path mapping to file object
+  private LinkedHashMap<String, DataFile> dataFiles = new LinkedHashMap<>();
+  private CarbonFile tableStatusFile;
+  private CarbonFile schemaFile;
+
+  private PrintStream out;
+
+  FileCollector(PrintStream out) {
+    this.out = out;
+  }
+
+  void collectFiles(String dataFolder) throws IOException {
+    Set<String> shards = new HashSet<>();
+    CarbonFile folder = FileFactory.getCarbonFile(dataFolder);
+    List<CarbonFile> files = folder.listFiles(true);
+    List<DataFile> unsortedFiles = new ArrayList<>();
+    for (CarbonFile file : files) {
+      if (isColumnarFile(file.getName())) {
+        DataFile dataFile = new DataFile(file);
+        dataFile.collectAllMeta();
+        unsortedFiles.add(dataFile);
+        collectNum(dataFile.getFooter());
+        shards.add(dataFile.getShardName());
+        totalDataSize += file.getSize();
+      } else if (file.getName().endsWith(CarbonTablePath.TABLE_STATUS_FILE)) {
+        tableStatusFile = file;
+      } else if (file.getName().startsWith(CarbonTablePath.SCHEMA_FILE)) {
+        schemaFile = file;
+      } else if (isStreamFile(file.getName())) {
+        out.println("WARN: input path contains streaming file, this tool does not support it yet, "
+            + "skipping it...");
+      }
+    }
+    unsortedFiles.sort((o1, o2) -> {
+      if (o1.getShardName().equalsIgnoreCase(o2.getShardName())) {
+        return Integer.parseInt(o1.getPartNo()) - Integer.parseInt(o2.getPartNo());
+      } else {
+        return o1.getShardName().compareTo(o2.getShardName());
+      }
+    });
+    for (DataFile collectedFile : unsortedFiles) {
+      this.dataFiles.put(collectedFile.getFilePath(), collectedFile);
+    }
+    numShard = shards.size();
+  }
+
+  private void collectNum(FileFooter3 footer) {
+    numBlock++;
+    numBlocklet += footer.blocklet_index_list.size();
+    numRow += footer.num_rows;
+    for (BlockletInfo3 blockletInfo3 : footer.blocklet_info_list3) {
+      numPage += blockletInfo3.number_number_of_pages;
+    }
+  }
+
+  private boolean isColumnarFile(String fileName) {
+    // if the timestamp in file name is "0", it is a streaming file
+    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
+        !CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
+  }
+
+  private boolean isStreamFile(String fileName) {
+    // if the timestamp in file name is "0", it is a streaming file
+    return fileName.endsWith(CarbonTablePath.CARBON_DATA_EXT) &&
+        CarbonTablePath.DataFileUtil.getTimeStampFromFileName(fileName).equals("0");
+  }
+
+  // return file path mapping to file object
+  LinkedHashMap<String, DataFile> getDataFiles() {
+    return dataFiles;
+  }
+
+  CarbonFile getTableStatusFile() {
+    return tableStatusFile;
+  }
+
+  CarbonFile getSchemaFile() {
+    return schemaFile;
+  }
+
+  int getNumDataFiles() {
+    return dataFiles.size();
+  }
+
+  void printBasicStats() {
+    if (dataFiles.size() == 0) {
+      System.out.println("no data file found");
+      return;
+    }
+    out.println("## Summary");
+    out.println(
+        String.format("total: %,d blocks, %,d shards, %,d blocklets, %,d pages, %,d rows, %s",
+            numBlock, numShard, numBlocklet, numPage, numRow, Strings.formatSize(totalDataSize)));
+    out.println(
+        String.format("avg: %s/block, %s/blocklet, %,d rows/block, %,d rows/blocklet",
+            Strings.formatSize((float) totalDataSize / numBlock),
+            Strings.formatSize((float) totalDataSize / numBlocklet),
+            numRow / numBlock,
+            numRow / numBlocklet));
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java b/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java
new file mode 100644
index 0000000..805d4e8
--- /dev/null
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/ScanBenchmark.java
@@ -0,0 +1,205 @@
+/*
+ * 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.tool;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.core.datastore.block.BlockletInfos;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
+import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
+import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.util.DataFileFooterConverterV3;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.FileFooter3;
+import org.apache.carbondata.format.FileHeader;
+
+import org.apache.commons.cli.CommandLine;
+
+class ScanBenchmark implements Command {
+
+  private String dataFolder;
+  private PrintStream out;
+  private DataFile file;
+
+  ScanBenchmark(String dataFolder, PrintStream out) {
+    this.dataFolder = dataFolder;
+    this.out = out;
+  }
+
+  @Override
+  public void run(CommandLine line) throws IOException, MemoryException {
+    if (line.hasOption("f")) {
+      String filePath = line.getOptionValue("f");
+      file = new DataFile(FileFactory.getCarbonFile(filePath));
+    } else {
+      FileCollector collector = new FileCollector(out);
+      collector.collectFiles(dataFolder);
+      if (collector.getNumDataFiles() == 0) {
+        return;
+      }
+      Map<String, DataFile> dataFiles = collector.getDataFiles();
+      file = dataFiles.entrySet().iterator().next().getValue();
+    }
+
+    out.println("\n## Benchmark");
+    AtomicReference<FileHeader> fileHeaderRef = new AtomicReference<>();
+    AtomicReference<FileFooter3> fileFoorterRef = new AtomicReference<>();
+    AtomicReference<DataFileFooter> convertedFooterRef = new AtomicReference<>();
+
+    // benchmark read header and footer time
+    benchmarkOperation("ReadHeaderAndFooter", () -> {
+      fileHeaderRef.set(file.readHeader());
+      fileFoorterRef.set(file.readFooter());
+    });
+    FileHeader fileHeader = fileHeaderRef.get();
+    FileFooter3 fileFooter = fileFoorterRef.get();
+
+    // benchmark convert footer
+    benchmarkOperation("ConvertFooter", () -> {
+      convertFooter(fileHeader, fileFooter);
+    });
+
+    // benchmark read all meta and convert footer
+    benchmarkOperation("ReadAllMetaAndConvertFooter", () -> {
+      DataFileFooter footer = readAndConvertFooter(file);
+      convertedFooterRef.set(footer);
+    });
+
+    if (line.hasOption("c")) {
+      String columnName = line.getOptionValue("c");
+      out.println("\nScan column '" + columnName + "'");
+
+      DataFileFooter footer = convertedFooterRef.get();
+      AtomicReference<AbstractRawColumnChunk> columnChunk = new AtomicReference<>();
+      int columnIndex = file.getColumnIndex(columnName);
+      boolean dimension = file.getColumn(columnName).isDimensionColumn();
+      for (int i = 0; i < footer.getBlockletList().size(); i++) {
+        int blockletId = i;
+        out.println(String.format("Blocklet#%d: total size %s, %,d pages, %,d rows",
+            blockletId,
+            Strings.formatSize(file.getColumnDataSizeInBytes(blockletId, columnIndex)),
+            footer.getBlockletList().get(blockletId).getNumberOfPages(),
+            footer.getBlockletList().get(blockletId).getNumberOfRows()));
+        benchmarkOperation("\tColumnChunk IO", () -> {
+          columnChunk.set(readBlockletColumnChunkIO(footer, blockletId, columnIndex, dimension));
+        });
+
+        if (dimensionColumnChunkReader != null) {
+          benchmarkOperation("\tDecompress Pages", () -> {
+            decompressDimensionPages(columnChunk.get(),
+                footer.getBlockletList().get(blockletId).getNumberOfPages());
+          });
+        } else {
+          benchmarkOperation("\tDecompress Pages", () -> {
+            decompressMeasurePages(columnChunk.get(),
+                footer.getBlockletList().get(blockletId).getNumberOfPages());
+          });
+        }
+      }
+    }
+
+  }
+
+  interface Operation {
+    void run() throws IOException, MemoryException;
+  }
+
+  private void benchmarkOperation(String opName, Operation op) throws IOException, MemoryException {
+    long start, end;
+    start = System.nanoTime();
+    op.run();
+    end = System.nanoTime();
+    out.println(String.format("%s takes %,d us", opName, (end - start) / 1000));
+  }
+
+  private DataFileFooter readAndConvertFooter(DataFile file) throws IOException {
+    int numBlocklets = file.getNumBlocklet();
+    BlockletInfos blockletInfos = new BlockletInfos(numBlocklets, 0, numBlocklets);
+    String segmentId = CarbonTablePath.DataFileUtil.getSegmentNo(file.getFilePath());
+    TableBlockInfo blockInfo =
+        new TableBlockInfo(file.getFilePath(), file.getFooterOffset(),
+            segmentId, new String[]{"localhost"}, file.getFileSizeInBytes(),
+            blockletInfos, ColumnarFormatVersion.V3, new String[0]);
+
+    DataFileFooterConverterV3 converter = new DataFileFooterConverterV3();
+    return converter.readDataFileFooter(blockInfo);
+  }
+
+  private DataFileFooter convertFooter(FileHeader fileHeader, FileFooter3 fileFooter) {
+    DataFileFooterConverterV3 converter = new DataFileFooterConverterV3();
+    return converter.convertDataFileFooter(fileHeader, fileFooter);
+  }
+
+  private DimensionColumnChunkReader dimensionColumnChunkReader;
+  private MeasureColumnChunkReader measureColumnChunkReader;
+
+  private AbstractRawColumnChunk readBlockletColumnChunkIO(
+      DataFileFooter footer, int blockletId, int columnIndex, boolean dimension)
+      throws IOException {
+    BlockletInfo blockletInfo = footer.getBlockletList().get(blockletId);
+    if (dimension) {
+      dimensionColumnChunkReader = CarbonDataReaderFactory.getInstance()
+          .getDimensionColumnChunkReader(ColumnarFormatVersion.V3, blockletInfo,
+              footer.getSegmentInfo().getColumnCardinality(), file.getFilePath(), false);
+      return dimensionColumnChunkReader.readRawDimensionChunk(file.getFileReader(), columnIndex);
+    } else {
+      columnIndex = columnIndex - file.numDimensions();
+      assert (columnIndex >= 0);
+      measureColumnChunkReader = CarbonDataReaderFactory.getInstance()
+          .getMeasureColumnChunkReader(ColumnarFormatVersion.V3, blockletInfo,
+              file.getFilePath(), false);
+      return measureColumnChunkReader.readRawMeasureChunk(file.getFileReader(), columnIndex);
+    }
+  }
+
+  private DimensionColumnPage[] decompressDimensionPages(
+      AbstractRawColumnChunk rawColumnChunk, int numPages) throws IOException, MemoryException {
+    DimensionColumnPage[] pages = new DimensionColumnPage[numPages];
+    for (int i = 0; i < pages.length; i++) {
+      pages[i] = dimensionColumnChunkReader.decodeColumnPage(
+          (DimensionRawColumnChunk) rawColumnChunk, i);
+    }
+    return pages;
+  }
+
+  private ColumnPage[] decompressMeasurePages(
+      AbstractRawColumnChunk rawColumnChunk, int numPages) throws IOException, MemoryException {
+    ColumnPage[] pages = new ColumnPage[numPages];
+    for (int i = 0; i < pages.length; i++) {
+      pages[i] = measureColumnChunkReader.decodeColumnPage(
+          (MeasureRawColumnChunk) rawColumnChunk, i);
+    }
+    return pages;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e07df44a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
----------------------------------------------------------------------
diff --git a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
index 0d0d6b5..4dc34c4 100644
--- a/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
+++ b/tools/cli/src/test/java/org/apache/carbondata/tool/CarbonCliTest.java
@@ -45,9 +45,9 @@ public class CarbonCliTest {
     fields[0] = new Field("name", DataTypes.STRING);
     fields[1] = new Field("age", DataTypes.INT);
 
-    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"age"},
+    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"},
         true, 3, 8, true);
-    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"age"},
+    TestUtil.writeFilesAndVerify(5000000, new Schema(fields), path, new String[]{"name"},
         true, 3, 8, true);
   }
 
@@ -69,7 +69,7 @@ public class CarbonCliTest {
   }
 
   @Test
-  public void testOutputIndividual() {
+  public void testSummaryOutputIndividual() {
     String[] args = {"-cmd", "summary", "-p", path};
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     PrintStream stream = new PrintStream(out);
@@ -79,20 +79,19 @@ public class CarbonCliTest {
         output.contains(
             "Input Folder: ./CarbonCliTest\n"
           + "## Summary\n"
-          + "total: 6 blocks, 2 shards, 12 blocklets, 314 pages, 10,000,000 rows, 30.72MB\n"
-          + "avg: 5.12MB/block, 2.56MB/blocklet, 1,666,666 rows/block, 833,333 rows/blocklet"));
+          + "total: 6 blocks, 2 shards, 14 blocklets, 314 pages, 10,000,000 rows, 32.26MB\n"
+          + "avg: 5.38MB/block, 2.30MB/blocklet, 1,666,666 rows/block, 714,285 rows/blocklet"));
 
     String[] args2 = {"-cmd", "summary", "-p", path, "-s"};
     out = new ByteArrayOutputStream();
     stream = new PrintStream(out);
     CarbonCli.run(args2, stream);
     output = new String(out.toByteArray());
-
     Assert.assertTrue(
         output.contains(
             "Column Name  Data Type  Column Type  SortColumn  Encoding          Ordinal  Id  \n"
-          + "age          INT        dimension    true        [INVERTED_INDEX]  1        NA  \n"
-          + "name         STRING     dimension    false       [INVERTED_INDEX]  0        NA  \n"));
+          + "name         STRING     dimension    true        [INVERTED_INDEX]  0        NA  \n"
+          + "age          INT        measure      false       []                1        NA  "));
 
     String[] args3 = {"-cmd", "summary", "-p", path, "-t"};
     out = new ByteArrayOutputStream();
@@ -113,38 +112,37 @@ public class CarbonCliTest {
     stream = new PrintStream(out);
     CarbonCli.run(args4, stream);
     output = new String(out.toByteArray());
-
     Assert.assertTrue(
         output.contains(
-            "BLK  BLKLT  NumPages  NumRows  Size    \n"
-          + "0    0      29        928,000  2.60MB  \n"
-          + "0    1      29        928,000  2.60MB  \n"
-          + "1    0      29        928,000  2.60MB  \n"
-          + "1    1      29        928,000  2.60MB  \n"
-          + "2    0      22        704,000  2.54MB  \n"
-          + "2    1      19        584,000  2.43MB  "));
+            "BLK  BLKLT  NumPages  NumRows  Size      \n"
+          + "0    0      25        800,000  2.58MB    \n"
+          + "0    1      25        800,000  2.58MB    \n"
+          + "1    0      25        800,000  2.58MB    \n"
+          + "1    1      25        800,000  2.58MB    \n"
+          + "2    0      25        800,000  2.58MB    \n"
+          + "2    1      25        800,000  2.58MB    \n"
+          + "2    2      7         200,000  660.74KB  "));
 
     String[] args5 = {"-cmd", "summary", "-p", path, "-c", "name"};
     out = new ByteArrayOutputStream();
     stream = new PrintStream(out);
     CarbonCli.run(args5, stream);
     output = new String(out.toByteArray());
-
     Assert.assertTrue(
         output.contains(
             "BLK  BLKLT  Meta Size  Data Size  LocalDict  DictEntries  DictSize  AvgPageSize  Min%    Max%    \n"
-          + "0    0      1.82KB     5.19MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "0    1      1.82KB     2.60MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "1    0      1.82KB     5.19MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "1    1      1.82KB     2.60MB     false      0            0.0B      11.96KB      robot0  robot9  \n"
-          + "2    0      1.38KB     4.97MB     false      0            0.0B      11.92KB      robot0  robot9  \n"
-          + "2    1      1.19KB     2.43MB     false      0            0.0B      11.42KB      robot0  robot9  \n"));
+          + "0    0      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot0  robot1  \n"
+          + "0    1      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot1  robot3  \n"
+          + "1    0      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot3  robot4  \n"
+          + "1    1      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot4  robot6  \n"
+          + "2    0      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot6  robot7  \n"
+          + "2    1      1.72KB     295.89KB   false      0            0.0B      11.77KB      robot8  robot9  \n"
+          + "2    2      492.0B     74.03KB    false      0            0.0B      10.51KB      robot9  robot9  "));
   }
 
   @Test
-  public void testOutputAll() {
+  public void testSummaryOutputAll() {
     String[] args = {"-cmd", "summary", "-p", path, "-a", "-c", "age"};
-
     ByteArrayOutputStream out = new ByteArrayOutputStream();
     PrintStream stream = new PrintStream(out);
     CarbonCli.run(args, stream);
@@ -153,14 +151,14 @@ public class CarbonCliTest {
         output.contains(
             "Input Folder: ./CarbonCliTest\n"
           + "## Summary\n"
-          + "total: 6 blocks, 2 shards, 12 blocklets, 314 pages, 10,000,000 rows, 30.72MB\n"
-          + "avg: 5.12MB/block, 2.56MB/blocklet, 1,666,666 rows/block, 833,333 rows/blocklet"));
+          + "total: 6 blocks, 2 shards, 14 blocklets, 314 pages, 10,000,000 rows, 32.26MB\n"
+          + "avg: 5.38MB/block, 2.30MB/blocklet, 1,666,666 rows/block, 714,285 rows/blocklet\n"));
 
     Assert.assertTrue(
         output.contains(
             "Column Name  Data Type  Column Type  SortColumn  Encoding          Ordinal  Id  \n"
-          + "age          INT        dimension    true        [INVERTED_INDEX]  1        NA  \n"
-          + "name         STRING     dimension    false       [INVERTED_INDEX]  0        NA  \n"));
+          + "name         STRING     dimension    true        [INVERTED_INDEX]  0        NA  \n"
+          + "age          INT        measure      false       []                1        NA  \n"));
 
     Assert.assertTrue(
         output.contains(
@@ -172,23 +170,35 @@ public class CarbonCliTest {
 
     Assert.assertTrue(
         output.contains(
-            "BLK  BLKLT  NumPages  NumRows  Size    \n"
-          + "0    0      29        928,000  2.60MB  \n"
-          + "0    1      29        928,000  2.60MB  \n"
-          + "1    0      29        928,000  2.60MB  \n"
-          + "1    1      29        928,000  2.60MB  \n"
-          + "2    0      22        704,000  2.54MB  \n"
-          + "2    1      19        584,000  2.43MB  "));
+            "BLK  BLKLT  NumPages  NumRows  Size      \n"
+          + "0    0      25        800,000  2.58MB    \n"
+          + "0    1      25        800,000  2.58MB    \n"
+          + "1    0      25        800,000  2.58MB    \n"
+          + "1    1      25        800,000  2.58MB    \n"
+          + "2    0      25        800,000  2.58MB    \n"
+          + "2    1      25        800,000  2.58MB    \n"
+          + "2    2      7         200,000  660.74KB  "));
 
     Assert.assertTrue(
         output.contains(
           "BLK  BLKLT  Meta Size  Data Size  LocalDict  DictEntries  DictSize  AvgPageSize  Min%  Max%   \n"
-        + "0    0      1.81KB     2.26MB     false      0            0.0B      79.61KB      0.0   15.5   \n"
-        + "0    1      1.81KB     2.26MB     false      0            0.0B      79.60KB      15.5  30.9   \n"
-        + "1    0      1.81KB     2.26MB     false      0            0.0B      79.62KB      30.9  46.4   \n"
-        + "1    1      1.81KB     2.26MB     false      0            0.0B      79.60KB      46.4  61.9   \n"
-        + "2    0      1.37KB     2.28MB     false      0            0.0B      106.11KB     61.9  80.5   \n"
-        + "2    1      1.19KB     2.22MB     false      0            0.0B      119.55KB     80.5  100.0  "));
+        + "0    0      2.90KB     4.87MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "0    1      2.90KB     2.29MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "1    0      2.90KB     4.87MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "1    1      2.90KB     2.29MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "2    0      2.90KB     5.52MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "2    1      2.90KB     2.94MB     false      0            0.0B      93.76KB      0.0   100.0  \n"
+        + "2    2      830.0B     586.81KB   false      0            0.0B      83.71KB      0.0   100.0 "));
+  }
+
+  @Test
+  public void testBenchmark() {
+    String[] args = {"-cmd", "benchmark", "-p", path, "-a", "-c", "name"};
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    PrintStream stream = new PrintStream(out);
+    CarbonCli.run(args, stream);
+    String output = new String(out.toByteArray());
+    System.out.println(output);
   }
 
   @After