You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by do...@apache.org on 2022/01/25 00:20:08 UTC

[orc] branch main updated: ORC-1098: [C++] Support specifying type ids or column names in cpp tools (#1020)

This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/main by this push:
     new 89af2cb  ORC-1098: [C++] Support specifying type ids or column names in cpp tools (#1020)
89af2cb is described below

commit 89af2cbe53fe4e53c948a4c389bdd14da3303687
Author: Quanlong Huang <hu...@gmail.com>
AuthorDate: Tue Jan 25 08:20:01 2022 +0800

    ORC-1098: [C++] Support specifying type ids or column names in cpp tools (#1020)
    
    ### What changes were proposed in this pull request?
    
    This is a follow-up task of #921. Currently we have options for the tools to work on specified top-level column fields. However, ACID ORC files usually have nested structure. We need the type ids to specify nested columns. As an extension, adding support for column names will also be helpful. So we don't need to manually convert column names to type ids. Also reports the valid values when an invalid column name is given.
    
    This PR extracts the option parsing codes into ToolsHelper. So similiar cpp tools can share the same option set.
    
    ### Why are the changes needed?
    
    It makes the tools more useful in practice.
    
    ### How was this patch tested?
    
    Added unit tests for the new options.
---
 c++/src/Reader.cc              | 10 ++++-
 site/_docs/cpp-tools.md        | 45 +++++++++++++++++++--
 tools/src/CMakeLists.txt       |  4 ++
 tools/src/FileContents.cc      | 50 +++++++-----------------
 tools/src/FileMemory.cc        | 79 ++++++++++++++-----------------------
 tools/src/FileScan.cc          | 75 ++++++-----------------------------
 tools/src/ToolsHelper.cc       | 89 ++++++++++++++++++++++++++++++++++++++++++
 tools/src/ToolsHelper.hh       | 25 ++++++++++++
 tools/test/TestFileContents.cc | 79 ++++++++++++++++++++++++++++++++++++-
 tools/test/TestFileScan.cc     | 66 ++++++++++++++++++++++++++-----
 10 files changed, 358 insertions(+), 164 deletions(-)

diff --git a/c++/src/Reader.cc b/c++/src/Reader.cc
index 8ac0677..13722df 100644
--- a/c++/src/Reader.cc
+++ b/c++/src/Reader.cc
@@ -216,7 +216,15 @@ namespace orc {
     if (ite != nameIdMap.end()) {
       updateSelectedByTypeId(selectedColumns, ite->second);
     } else {
-      throw ParseError("Invalid column selected " + fieldName);
+      bool first = true;
+      std::ostringstream ss;
+      ss << "Invalid column selected " << fieldName << ". Valid names are ";
+      for (auto it = nameIdMap.begin(); it != nameIdMap.end(); ++it) {
+        if (!first) ss << ", ";
+        ss << it->first;
+        first = false;
+      }
+      throw ParseError(ss.str());
     }
   }
 
diff --git a/site/_docs/cpp-tools.md b/site/_docs/cpp-tools.md
index d4d6e75..786c3ed 100644
--- a/site/_docs/cpp-tools.md
+++ b/site/_docs/cpp-tools.md
@@ -10,7 +10,13 @@ Displays the contents of the ORC file as a JSON document. With the
 `columns` argument only the selected columns are printed.
 
 ~~~ shell
-% orc-contents  [--columns=1,2,...] <filename>
+% orc-contents [options] <filename>
+Options:
+	-h --help
+	-c --columns		Comma separated list of top-level column fields
+	-t --columnTypeIds	Comma separated list of column type ids
+	-n --columnNames	Comma separated list of column names
+	-b --batch		Batch size for reading
 ~~~
 
 If you run it on the example file TestOrcFile.test1.orc, you'll see (without
@@ -115,7 +121,13 @@ to set the batch size which is 1024 rows by default. It is useful to check
 if the ORC file is damaged.
 
 ~~~ shell
-% orc-scan [--batch=<size>] <filename>
+% orc-scan [options] <filename>...
+Options:
+	-h --help
+	-c --columns		Comma separated list of top-level column fields
+	-t --columnTypeIds	Comma separated list of column type ids
+	-n --columnNames	Comma separated list of column names
+	-b --batch		Batch size for reading
 ~~~
 
 If you run it on the example file TestOrcFile.test1.orc, you'll see:
@@ -135,7 +147,7 @@ With the `withIndex` option to include column statistics in each row group.
 % orc-statistics [--withIndex] <filename>
 ~~~
 
-If you run it on the example file TestOrcFile.TestOrcFile.columnProjection.orc
+If you run it on the example file TestOrcFile.columnProjection.orc
 you'll see:
 
 ~~~ shell
@@ -265,4 +277,29 @@ Has null: no
 Minimum: 1059d81c9025a217
 Maximum: ffc17f0e35e1a6c0
 Total length: 15941
-~~~
\ No newline at end of file
+~~~
+
+## orc-memory
+
+Estimate the memory footprint for reading the ORC file.
+
+~~~ shell
+% orc-memory [options] <filename>
+Options:
+	-h --help
+	-c --columns		Comma separated list of top-level column fields
+	-t --columnTypeIds	Comma separated list of column type ids
+	-n --columnNames	Comma separated list of column names
+	-b --batch		Batch size for reading
+~~~
+
+If you run it on the example file TestOrcFile.columnProjection.orc
+you'll see:
+
+~~~ shell
+% orc-memory examples/TestOrcFile.columnProjection.orc,
+Reader memory estimate: 202972
+Batch memory estimate:  27000
+Total memory estimate:  229972
+Actual max memory used: 160381
+~~~
diff --git a/tools/src/CMakeLists.txt b/tools/src/CMakeLists.txt
index 6cfb950..d55581b 100644
--- a/tools/src/CMakeLists.txt
+++ b/tools/src/CMakeLists.txt
@@ -28,6 +28,7 @@ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -g ${CXX11_FLAGS} ${WARN_FLAGS}")
 
 add_executable (orc-contents
   FileContents.cc
+  ToolsHelper.cc
   )
 
 target_link_libraries (orc-contents
@@ -37,6 +38,7 @@ target_link_libraries (orc-contents
 
 add_executable (orc-scan
   FileScan.cc
+  ToolsHelper.cc
   )
 
 target_link_libraries (orc-scan
@@ -46,6 +48,7 @@ target_link_libraries (orc-scan
 
 add_executable (orc-metadata
   FileMetadata.cc
+  ToolsHelper.cc
   )
 
 target_link_libraries (orc-metadata
@@ -65,6 +68,7 @@ target_link_libraries (orc-statistics
 
 add_executable (orc-memory
   FileMemory.cc
+  ToolsHelper.cc
   )
 
 target_link_libraries (orc-memory
diff --git a/tools/src/FileContents.cc b/tools/src/FileContents.cc
index e7bfdac..6a225a8 100644
--- a/tools/src/FileContents.cc
+++ b/tools/src/FileContents.cc
@@ -16,14 +16,11 @@
  * limitations under the License.
  */
 
-#include "orc/orc-config.hh"
-#include "orc/ColumnPrinter.hh"
-#include "orc/Exceptions.hh"
+#include "ToolsHelper.hh"
 
 #include <memory>
 #include <string>
 #include <iostream>
-#include <string>
 
 void printContents(const char* filename, const orc::RowReaderOptions& rowReaderOpts) {
   orc::ReaderOptions readerOpts;
@@ -50,40 +47,23 @@ void printContents(const char* filename, const orc::RowReaderOptions& rowReaderO
 }
 
 int main(int argc, char* argv[]) {
-  if (argc < 2) {
-    std::cout << "Usage: orc-contents <filename> [--columns=1,2,...]\n"
-              << "Print contents of <filename>.\n"
-              << "If columns are specified, only these top-level (logical) columns are printed.\n" ;
+  uint64_t batchSize; // not used
+  orc::RowReaderOptions rowReaderOptions;
+  bool success = parseOptions(&argc, &argv, &batchSize, &rowReaderOptions);
+
+  if (argc < 1 || !success) {
+    std::cerr << "Usage: orc-contents [options] <filename>...\n";
+    printOptions(std::cerr);
+    std::cerr << "Print contents of ORC files.\n";
     return 1;
   }
-  try {
-    const std::string COLUMNS_PREFIX = "--columns=";
-    std::list<uint64_t> cols;
-    char* filename = ORC_NULLPTR;
-
-    // Read command-line options
-    char *param, *value;
-    for (int i = 1; i < argc; i++) {
-      if ( (param = std::strstr(argv[i], COLUMNS_PREFIX.c_str())) ) {
-        value = std::strtok(param+COLUMNS_PREFIX.length(), "," );
-        while (value) {
-          cols.push_back(static_cast<uint64_t>(std::atoi(value)));
-          value = std::strtok(ORC_NULLPTR, "," );
-        }
-      } else {
-        filename = argv[i];
-      }
+  for (int i = 0; i < argc; ++i) {
+    try {
+      printContents(argv[i], rowReaderOptions);
+    } catch (std::exception& ex) {
+      std::cerr << "Caught exception in " << argv[i] << ": " << ex.what() << "\n";
+      return 1;
     }
-    orc::RowReaderOptions rowReaderOpts;
-    if (cols.size() > 0) {
-      rowReaderOpts.include(cols);
-    }
-    if (filename != ORC_NULLPTR) {
-      printContents(filename, rowReaderOpts);
-    }
-  } catch (std::exception& ex) {
-    std::cerr << "Caught exception: " << ex.what() << "\n";
-    return 1;
   }
   return 0;
 }
diff --git a/tools/src/FileMemory.cc b/tools/src/FileMemory.cc
index 2ce9aba..af6ce0d 100644
--- a/tools/src/FileMemory.cc
+++ b/tools/src/FileMemory.cc
@@ -16,15 +16,12 @@
  * limitations under the License.
  */
 
-#include "orc/orc-config.hh"
-#include "orc/ColumnPrinter.hh"
-#include "orc/Exceptions.hh"
+#include "ToolsHelper.hh"
 
 #include <string>
 #include <memory>
 #include <iostream>
 #include <map>
-#include <exception>
 
 class TestMemoryPool: public orc::MemoryPool {
 private:
@@ -60,13 +57,9 @@ public:
 TestMemoryPool::~TestMemoryPool() {}
 
 void processFile(const char* filename,
-                 const std::list<uint64_t>& cols,
-                 uint32_t batchSize) {
+                 const orc::RowReaderOptions& rowReaderOpts,
+                 uint64_t batchSize) {
   orc::ReaderOptions readerOpts;
-  orc::RowReaderOptions rowReaderOpts;
-  if (cols.size() > 0) {
-    rowReaderOpts.include(cols);
-  }
   std::unique_ptr<orc::MemoryPool> pool(new TestMemoryPool());
   readerOpts.setMemoryPool(*(pool.get()));
 
@@ -76,7 +69,18 @@ void processFile(const char* filename,
 
   std::unique_ptr<orc::ColumnVectorBatch> batch =
       rowReader->createRowBatch(batchSize);
-  uint64_t readerMemory = reader->getMemoryUseByFieldId(cols);
+  uint64_t readerMemory;
+  if (rowReaderOpts.getIndexesSet()) {
+    readerMemory = reader->getMemoryUseByFieldId(rowReaderOpts.getInclude());
+  } else if (rowReaderOpts.getNamesSet()) {
+    readerMemory = reader->getMemoryUseByName(rowReaderOpts.getIncludeNames());
+  } else if (rowReaderOpts.getTypeIdsSet()) {
+    readerMemory = reader->getMemoryUseByTypeId(rowReaderOpts.getInclude());
+  } else {
+    // default is to select all columns
+    readerMemory = reader->getMemoryUseByName({});
+  }
+
   uint64_t batchMemory = batch->getMemoryUsage();
   while (rowReader->next(*batch)) {}
   uint64_t actualMemory =
@@ -93,47 +97,22 @@ void processFile(const char* filename,
 }
 
 int main(int argc, char* argv[]) {
-  if (argc < 2) {
-    std::cout << "Usage: orc-memory [--columns=column1,column2,...] "
-        << "[--batch=rows_in_batch] <filename> \n";
+  uint64_t batchSize = 1000;
+  orc::RowReaderOptions rowReaderOptions;
+  bool success = parseOptions(&argc, &argv, &batchSize, &rowReaderOptions);
+  if (argc < 1 || !success) {
+    std::cerr << "Usage: orc-memory [options] <filename>...\n";
+    printOptions(std::cerr);
+    std::cerr << "Estimate the memory footprint for reading ORC files\n";
     return 1;
   }
-
-  const std::string COLUMNS_PREFIX = "--columns=";
-  const std::string BATCH_PREFIX = "--batch=";
-  char* filename = ORC_NULLPTR;
-
-  // Default parameters
-  std::list<uint64_t> cols;
-  uint32_t batchSize = 1000;
-
-  // Read command-line options
-  char *param, *value;
-  for (int i = 1; i < argc; i++) {
-    if ( (param = std::strstr(argv[i], COLUMNS_PREFIX.c_str())) ) {
-      value = std::strtok(param+COLUMNS_PREFIX.length(), "," );
-      while (value) {
-        cols.push_back(static_cast<uint64_t>(std::atoi(value)));
-        value = std::strtok(ORC_NULLPTR, "," );
-      }
-    } else if ( (param=strstr(argv[i], BATCH_PREFIX.c_str())) ) {
-      batchSize =
-        static_cast<uint32_t>(std::atoi(param+BATCH_PREFIX.length()));
-    } else {
-      filename = argv[i];
+  for (int i = 0; i < argc; ++i) {
+    try {
+      processFile(argv[i], rowReaderOptions, batchSize);
+    } catch (std::exception& ex) {
+      std::cerr << "Caught exception: " << ex.what() << "\n";
+      return 1;
     }
   }
-
-  if (filename == ORC_NULLPTR) {
-    std::cout << "Error: Filename not provided.\n";
-    return 1;
-  }
-
-  try {
-    processFile(filename, cols, batchSize);
-    return 0;
-  } catch (std::exception& ex) {
-    std::cerr << "Caught exception: " << ex.what() << "\n";
-    return 1;
-  }
+  return 0;
 }
diff --git a/tools/src/FileScan.cc b/tools/src/FileScan.cc
index a212dc0..1e87883 100644
--- a/tools/src/FileScan.cc
+++ b/tools/src/FileScan.cc
@@ -16,15 +16,9 @@
  * limitations under the License.
  */
 
-#include "orc/ColumnPrinter.hh"
+#include "ToolsHelper.hh"
 
-#include "orc/Exceptions.hh"
-
-#include <getopt.h>
-#include <string>
-#include <memory>
 #include <iostream>
-#include <string>
 
 void scanFile(std::ostream & out, const char* filename, uint64_t batchSize,
               const orc::RowReaderOptions& rowReaderOpts) {
@@ -46,64 +40,21 @@ void scanFile(std::ostream & out, const char* filename, uint64_t batchSize,
 }
 
 int main(int argc, char* argv[]) {
-  static struct option longOptions[] = {
-    {"help", no_argument, ORC_NULLPTR, 'h'},
-    {"batch", required_argument, ORC_NULLPTR, 'b'},
-    {"columns", required_argument, ORC_NULLPTR, 'c'},
-    {ORC_NULLPTR, 0, ORC_NULLPTR, 0}
-  };
-  bool helpFlag = false;
   uint64_t batchSize = 1024;
-  std::list<uint64_t> cols;
   orc::RowReaderOptions rowReaderOptions;
-  int opt;
-  char *tail;
-  do {
-    opt = getopt_long(argc, argv, "hb:c:", longOptions, ORC_NULLPTR);
-    switch (opt) {
-    case '?':
-    case 'h':
-      helpFlag = true;
-      opt = -1;
-      break;
-    case 'b':
-      batchSize = strtoul(optarg, &tail, 10);
-      if (*tail != '\0') {
-        fprintf(stderr, "The --batch parameter requires an integer option.\n");
-        return 1;
-      }
-      break;
-    case 'c': {
-      char *col = std::strtok(optarg, ",");
-      while (col) {
-        cols.push_back(static_cast<uint64_t>(std::atoi(col)));
-        col = std::strtok(ORC_NULLPTR, ",");
-      }
-      if (!cols.empty()) {
-        rowReaderOptions.include(cols);
-      }
-      break;
-    }
-    default: break;
-    }
-  } while (opt != -1);
-  argc -= optind;
-  argv += optind;
-
-  if (argc < 1 || helpFlag) {
-    std::cerr << "Usage: orc-scan [-h] [--help]\n"
-              << "                [-c 1,2,...] [--columns=1,2,...]\n"
-              << "                [-b<size>] [--batch=<size>] <filename>\n";
+  bool success = parseOptions(&argc, &argv, &batchSize, &rowReaderOptions);
+  if (argc < 1 || !success) {
+    std::cerr << "Usage: orc-scan [options] <filename>...\n";
+    printOptions(std::cerr);
+    std::cerr << "Scans and displays the row count of the ORC files.\n";
     return 1;
-  } else {
-    for(int i=0; i < argc; ++i) {
-      try {
-        scanFile(std::cout, argv[i], batchSize, rowReaderOptions);
-      } catch (std::exception& ex) {
-        std::cerr << "Caught exception in " << argv[i]
-                  << ": " << ex.what() << "\n";
-        return 1;
-      }
+  }
+  for (int i = 0; i < argc; ++i) {
+    try {
+      scanFile(std::cout, argv[i], batchSize, rowReaderOptions);
+    } catch (std::exception& ex) {
+      std::cerr << "Caught exception in " << argv[i] << ": " << ex.what() << "\n";
+      return 1;
     }
   }
   return 0;
diff --git a/tools/src/ToolsHelper.cc b/tools/src/ToolsHelper.cc
new file mode 100644
index 0000000..20204ab
--- /dev/null
+++ b/tools/src/ToolsHelper.cc
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#include "ToolsHelper.hh"
+
+#include <getopt.h>
+
+void printOptions(std::ostream& out) {
+  out << "Options:\n"
+      << "\t-h --help\n"
+      << "\t-c --columns\t\tComma separated list of top-level column fields\n"
+      << "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+      << "\t-n --columnNames\tComma separated list of column names\n"
+      << "\t-b --batch\t\tBatch size for reading\n";
+}
+
+bool parseOptions(int* argc, char** argv[], uint64_t* batchSize, orc::RowReaderOptions* rowReaderOpts) {
+  static struct option longOptions[] = {
+    {"help", no_argument, ORC_NULLPTR, 'h'},
+    {"batch", required_argument, ORC_NULLPTR, 'b'},
+    {"columns", required_argument, ORC_NULLPTR, 'c'},
+    {"columnTypeIds", required_argument, ORC_NULLPTR, 't'},
+    {"columnNames", required_argument, ORC_NULLPTR, 'n'},
+    {ORC_NULLPTR, 0, ORC_NULLPTR, 0}
+  };
+  std::list<uint64_t> cols;
+  std::list<std::string> colNames;
+  int opt;
+  char *tail;
+  do {
+    opt = getopt_long(*argc, *argv, "hb:c:t:n:", longOptions, ORC_NULLPTR);
+    switch (opt) {
+      case '?':
+      case 'h':
+        return false;
+      case 'b':
+        *batchSize = strtoul(optarg, &tail, 10);
+        if (*tail != '\0') {
+          fprintf(stderr, "The --batch parameter requires an integer option.\n");
+          return false;
+        }
+        break;
+      case 't':
+      case 'c':
+      case 'n': {
+        bool empty = true;
+        char *col = std::strtok(optarg, ",");
+        while (col) {
+          if (opt == 'n') {
+            colNames.emplace_back(col);
+          } else {
+            cols.emplace_back(static_cast<uint64_t>(std::atoi(col)));
+          }
+          empty = false;
+          col = std::strtok(ORC_NULLPTR, ",");
+        }
+        if (!empty) {
+          if (opt == 'c') {
+            rowReaderOpts->include(cols);
+          } else if (opt == 't') {
+            rowReaderOpts->includeTypes(cols);
+          } else {
+            rowReaderOpts->include(colNames);
+          }
+        }
+        break;
+      }
+      default: break;
+    }
+  } while (opt != -1);
+  *argc -= optind;
+  *argv += optind;
+  return true;
+}
diff --git a/tools/src/ToolsHelper.hh b/tools/src/ToolsHelper.hh
new file mode 100644
index 0000000..a15235a
--- /dev/null
+++ b/tools/src/ToolsHelper.hh
@@ -0,0 +1,25 @@
+/**
+ * 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.
+ */
+
+#include "orc/ColumnPrinter.hh"
+#include <iostream>
+
+void printOptions(std::ostream& out);
+
+bool parseOptions(int* argc, char** argv[], uint64_t* batchSize,
+                  orc::RowReaderOptions* rowReaderOpts);
diff --git a/tools/test/TestFileContents.cc b/tools/test/TestFileContents.cc
index 9c3a6d5..9b942f1 100644
--- a/tools/test/TestFileContents.cc
+++ b/tools/test/TestFileContents.cc
@@ -55,7 +55,9 @@ TEST (TestFileContents, testRaw) {
 TEST (TestFileContents, testSelectedColumns) {
   const std::string pgm = findProgram("tools/src/orc-contents");
   const std::string file = findExample("TestOrcFile.test1.orc");
-  const std::string options = "--columns=1,3,5,7";
+  const std::string columnFields = "1,3,5,7";
+  const std::string columnTypeIds = "2,4,6,8";
+  const std::string columnNames = "byte1,int1,float1,bytes1";
   const std::string expected =
     "{\"byte1\": 1, \"int1\": 65536, \"float1\": 1, \"bytes1\": [0, 1, 2, 3, 4]}\n"
       "{\"byte1\": 100, \"int1\": 65536, \"float1\": 2, \"bytes1\": []}\n";
@@ -63,7 +65,80 @@ TEST (TestFileContents, testSelectedColumns) {
   std::string output;
   std::string error;
 
-  EXPECT_EQ(0, runProgram({pgm, options, file}, output, error));
+  EXPECT_EQ(0, runProgram({pgm, "--columns=" + columnFields, file}, output, error));
   EXPECT_EQ(expected, output);
   EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "--columns", columnFields, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "-c", columnFields, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+
+  EXPECT_EQ(0, runProgram({pgm, "--columnTypeIds=" + columnTypeIds, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "--columnTypeIds", columnTypeIds, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "-t", columnTypeIds, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+
+  EXPECT_EQ(0, runProgram({pgm, "--columnNames=" + columnNames, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "--columnNames", columnNames, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "-n", columnNames, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+}
+
+TEST (TestFileContents, testNestedColumns) {
+  const std::string pgm = findProgram("tools/src/orc-contents");
+  const std::string file = findExample("complextypes_iceberg.orc");
+  const std::string columnTypeIds = "1,15,16";
+  const std::string columnNames = "id,nested_struct.a,nested_struct.b";
+  const std::string expected =
+    "{\"id\": 8, \"nested_struct\": {\"a\": -1, \"b\": [-1]}}\n";
+
+  std::string output;
+  std::string error;
+
+  EXPECT_EQ(0, runProgram({pgm, "--columnTypeIds=" + columnTypeIds, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "--columnTypeIds", columnTypeIds, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "-t", columnTypeIds, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+
+  EXPECT_EQ(0, runProgram({pgm, "--columnNames=" + columnNames, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "--columnNames", columnNames, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+  EXPECT_EQ(0, runProgram({pgm, "-n", columnNames, file}, output, error));
+  EXPECT_EQ(expected, output);
+  EXPECT_EQ("", error);
+}
+
+TEST (TestFileContents, testInvalidName) {
+  const std::string pgm = findProgram("tools/src/orc-contents");
+  const std::string file = findExample("TestOrcFile.test1.orc");
+  const std::string error_msg =
+    "Invalid column selected abc. Valid names are boolean1, byte1, bytes1, double1, "
+    "float1, int1, list, list.int1, list.string1, long1, map, map.int1, map.string1, "
+    "middle, middle.list, middle.list.int1, middle.list.string1, short1, string1";
+
+  std::string output;
+  std::string error;
+  EXPECT_EQ(1, runProgram({pgm, "-n", "byte1,abc", file}, output, error));
+  EXPECT_EQ("", output);
+  EXPECT_NE(std::string::npos, error.find(error_msg));
 }
diff --git a/tools/test/TestFileScan.cc b/tools/test/TestFileScan.cc
index b53841f..0aaa689 100644
--- a/tools/test/TestFileScan.cc
+++ b/tools/test/TestFileScan.cc
@@ -119,39 +119,85 @@ TEST (TestFileScan, testBadCommand) {
   EXPECT_EQ(1, runProgram({pgm, file, std::string("-b")}, output, error));
   EXPECT_EQ("", output);
   EXPECT_EQ("orc-scan: option requires an argument -- b\n"
-            "Usage: orc-scan [-h] [--help]\n"
-            "                [-c 1,2,...] [--columns=1,2,...]\n"
-            "                [-b<size>] [--batch=<size>] <filename>\n",
+            "Usage: orc-scan [options] <filename>...\n"
+            "Options:\n"
+            "\t-h --help\n"
+            "\t-c --columns\t\tComma separated list of top-level column fields\n"
+            "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+            "\t-n --columnNames\tComma separated list of column names\n"
+            "\t-b --batch\t\tBatch size for reading\n"
+            "Scans and displays the row count of the ORC files.\n",
             removeChars(stripPrefix(error, "orc-scan: "),"'`"));
 
   EXPECT_EQ(1, runProgram({pgm, file, std::string("-b"),
           std::string("20x")}, output, error));
   EXPECT_EQ("", output);
-  EXPECT_EQ("The --batch parameter requires an integer option.\n", error);
+  EXPECT_EQ("The --batch parameter requires an integer option.\n"
+            "Usage: orc-scan [options] <filename>...\n"
+            "Options:\n"
+            "\t-h --help\n"
+            "\t-c --columns\t\tComma separated list of top-level column fields\n"
+            "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+            "\t-n --columnNames\tComma separated list of column names\n"
+            "\t-b --batch\t\tBatch size for reading\n"
+            "Scans and displays the row count of the ORC files.\n",
+            error);
 
   EXPECT_EQ(1, runProgram({pgm, file, std::string("-b"),
           std::string("x30")}, output, error));
   EXPECT_EQ("", output);
-  EXPECT_EQ("The --batch parameter requires an integer option.\n", error);
+  EXPECT_EQ("The --batch parameter requires an integer option.\n"
+            "Usage: orc-scan [options] <filename>...\n"
+            "Options:\n"
+            "\t-h --help\n"
+            "\t-c --columns\t\tComma separated list of top-level column fields\n"
+            "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+            "\t-n --columnNames\tComma separated list of column names\n"
+            "\t-b --batch\t\tBatch size for reading\n"
+            "Scans and displays the row count of the ORC files.\n",
+            error);
 
   EXPECT_EQ(1, runProgram({pgm, file, std::string("--batch")},
                           output, error));
   EXPECT_EQ("", output);
   EXPECT_EQ("orc-scan: option --batch requires an argument\n"
-            "Usage: orc-scan [-h] [--help]\n"
-            "                [-c 1,2,...] [--columns=1,2,...]\n"
-            "                [-b<size>] [--batch=<size>] <filename>\n",
+            "Usage: orc-scan [options] <filename>...\n"
+            "Options:\n"
+            "\t-h --help\n"
+            "\t-c --columns\t\tComma separated list of top-level column fields\n"
+            "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+            "\t-n --columnNames\tComma separated list of column names\n"
+            "\t-b --batch\t\tBatch size for reading\n"
+            "Scans and displays the row count of the ORC files.\n",
             removeChars(stripPrefix(error, "orc-scan: "), "'`"));
 
   EXPECT_EQ(1, runProgram({pgm, file, std::string("--batch"),
           std::string("20x")}, output, error));
   EXPECT_EQ("", output);
-  EXPECT_EQ("The --batch parameter requires an integer option.\n", error);
+  EXPECT_EQ("The --batch parameter requires an integer option.\n"
+            "Usage: orc-scan [options] <filename>...\n"
+            "Options:\n"
+            "\t-h --help\n"
+            "\t-c --columns\t\tComma separated list of top-level column fields\n"
+            "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+            "\t-n --columnNames\tComma separated list of column names\n"
+            "\t-b --batch\t\tBatch size for reading\n"
+            "Scans and displays the row count of the ORC files.\n",
+            error);
 
   EXPECT_EQ(1, runProgram({pgm, file, std::string("--batch"),
             std::string("x30")}, output, error));
   EXPECT_EQ("", output);
-  EXPECT_EQ("The --batch parameter requires an integer option.\n", error);
+  EXPECT_EQ("The --batch parameter requires an integer option.\n"
+            "Usage: orc-scan [options] <filename>...\n"
+            "Options:\n"
+            "\t-h --help\n"
+            "\t-c --columns\t\tComma separated list of top-level column fields\n"
+            "\t-t --columnTypeIds\tComma separated list of column type ids\n"
+            "\t-n --columnNames\tComma separated list of column names\n"
+            "\t-b --batch\t\tBatch size for reading\n"
+            "Scans and displays the row count of the ORC files.\n",
+            error);
 }
 
 void checkForError(const std::string& filename, const std::string& error_msg) {