You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by we...@apache.org on 2017/04/21 14:22:03 UTC

parquet-cpp git commit: PARQUET-508: Add ParquetFilePrinter

Repository: parquet-cpp
Updated Branches:
  refs/heads/master 5ba8941de -> a54404ed0


PARQUET-508: Add ParquetFilePrinter

Author: Deepak Majeti <de...@hpe.com>

Closes #307 from majetideepak/PARQUET-508 and squashes the following commits:

de1364a [Deepak Majeti] add to api
1dc6260 [Deepak Majeti] PARQUET-508: Add ParquetFilePrinter


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

Branch: refs/heads/master
Commit: a54404ed0645e8f642cdebc2239cdd156070b20b
Parents: 5ba8941
Author: Deepak Majeti <de...@hpe.com>
Authored: Fri Apr 21 10:21:56 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Fri Apr 21 10:21:56 2017 -0400

----------------------------------------------------------------------
 CMakeLists.txt                  |   1 +
 src/parquet/api/reader.h        |   1 +
 src/parquet/file/CMakeLists.txt |   1 +
 src/parquet/file/printer.cc     | 143 +++++++++++++++++++++++++++++++++++
 src/parquet/file/printer.h      |  45 +++++++++++
 src/parquet/file/reader.cc      | 116 ----------------------------
 src/parquet/file/reader.h       |   3 -
 src/parquet/reader-test.cc      |  22 ++++--
 tools/parquet_reader.cc         |   3 +-
 9 files changed, 208 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 389c09c..b153d89 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -572,6 +572,7 @@ set(LIBPARQUET_SRCS
   src/parquet/compression.cc
 
   src/parquet/file/metadata.cc
+  src/parquet/file/printer.cc
   src/parquet/file/reader.cc
   src/parquet/file/reader-internal.cc
   src/parquet/file/writer.cc

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/api/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/api/reader.h b/src/parquet/api/reader.h
index 0bd4a24..f41a429 100644
--- a/src/parquet/api/reader.h
+++ b/src/parquet/api/reader.h
@@ -22,6 +22,7 @@
 #include "parquet/column/reader.h"
 #include "parquet/column/scan-all.h"
 #include "parquet/exception.h"
+#include "parquet/file/printer.h"
 #include "parquet/file/reader.h"
 
 // Metadata reader API

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/file/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/parquet/file/CMakeLists.txt b/src/parquet/file/CMakeLists.txt
index c31696b..82e7c80 100644
--- a/src/parquet/file/CMakeLists.txt
+++ b/src/parquet/file/CMakeLists.txt
@@ -17,6 +17,7 @@
 
 install(FILES
   metadata.h
+  printer.h
   reader.h
   writer.h
   DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/parquet/file")

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/file/printer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/printer.cc b/src/parquet/file/printer.cc
new file mode 100644
index 0000000..8dd9d55
--- /dev/null
+++ b/src/parquet/file/printer.cc
@@ -0,0 +1,143 @@
+// 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 "parquet/file/printer.h"
+
+#include <string>
+#include <vector>
+
+#include "parquet/column/scanner.h"
+
+using std::string;
+using std::vector;
+
+namespace parquet {
+// ----------------------------------------------------------------------
+// ParquetFilePrinter::DebugPrint
+
+// the fixed initial size is just for an example
+#define COL_WIDTH "30"
+
+void ParquetFilePrinter::DebugPrint(
+    std::ostream& stream, std::list<int> selected_columns, bool print_values) {
+  const FileMetaData* file_metadata = fileReader->metadata().get();
+
+  stream << "File statistics:\n";
+  stream << "Version: " << file_metadata->version() << "\n";
+  stream << "Created By: " << file_metadata->created_by() << "\n";
+  stream << "Total rows: " << file_metadata->num_rows() << "\n";
+  stream << "Number of RowGroups: " << file_metadata->num_row_groups() << "\n";
+  stream << "Number of Real Columns: "
+         << file_metadata->schema()->group_node()->field_count() << "\n";
+
+  if (selected_columns.size() == 0) {
+    for (int i = 0; i < file_metadata->num_columns(); i++) {
+      selected_columns.push_back(i);
+    }
+  } else {
+    for (auto i : selected_columns) {
+      if (i < 0 || i >= file_metadata->num_columns()) {
+        throw ParquetException("Selected column is out of range");
+      }
+    }
+  }
+
+  stream << "Number of Columns: " << file_metadata->num_columns() << "\n";
+  stream << "Number of Selected Columns: " << selected_columns.size() << "\n";
+  for (auto i : selected_columns) {
+    const ColumnDescriptor* descr = file_metadata->schema()->Column(i);
+    stream << "Column " << i << ": " << descr->name() << " ("
+           << TypeToString(descr->physical_type()) << ")" << std::endl;
+  }
+
+  for (int r = 0; r < file_metadata->num_row_groups(); ++r) {
+    stream << "--- Row Group " << r << " ---\n";
+
+    auto group_reader = fileReader->RowGroup(r);
+    std::unique_ptr<RowGroupMetaData> group_metadata = file_metadata->RowGroup(r);
+
+    stream << "--- Total Bytes " << group_metadata->total_byte_size() << " ---\n";
+    stream << "  rows: " << group_metadata->num_rows() << "---\n";
+
+    // Print column metadata
+    for (auto i : selected_columns) {
+      auto column_chunk = group_metadata->ColumnChunk(i);
+      std::shared_ptr<RowGroupStatistics> stats = column_chunk->statistics();
+
+      const ColumnDescriptor* descr = file_metadata->schema()->Column(i);
+      stream << "Column " << i << std::endl << ", values: " << column_chunk->num_values();
+      if (column_chunk->is_stats_set()) {
+        std::string min = stats->EncodeMin(), max = stats->EncodeMax();
+        stream << ", null values: " << stats->null_count()
+               << ", distinct values: " << stats->distinct_count() << std::endl
+               << "  max: " << FormatStatValue(descr->physical_type(), max.c_str())
+               << ", min: " << FormatStatValue(descr->physical_type(), min.c_str());
+      } else {
+        stream << "  Statistics Not Set";
+      }
+      stream << std::endl
+             << "  compression: " << CompressionToString(column_chunk->compression())
+             << ", encodings: ";
+      for (auto encoding : column_chunk->encodings()) {
+        stream << EncodingToString(encoding) << " ";
+      }
+      stream << std::endl
+             << "  uncompressed size: " << column_chunk->total_uncompressed_size()
+             << ", compressed size: " << column_chunk->total_compressed_size()
+             << std::endl;
+    }
+
+    if (!print_values) { continue; }
+
+    static constexpr int bufsize = 25;
+    char buffer[bufsize];
+
+    // Create readers for selected columns and print contents
+    vector<std::shared_ptr<Scanner>> scanners(selected_columns.size(), NULL);
+    int j = 0;
+    for (auto i : selected_columns) {
+      std::shared_ptr<ColumnReader> col_reader = group_reader->Column(i);
+
+      std::stringstream ss;
+      ss << "%-" << COL_WIDTH << "s";
+      std::string fmt = ss.str();
+
+      snprintf(buffer, bufsize, fmt.c_str(),
+          file_metadata->schema()->Column(i)->name().c_str());
+      stream << buffer;
+
+      // This is OK in this method as long as the RowGroupReader does not get
+      // deleted
+      scanners[j++] = Scanner::Make(col_reader);
+    }
+    stream << "\n";
+
+    bool hasRow;
+    do {
+      hasRow = false;
+      for (auto scanner : scanners) {
+        if (scanner->HasNext()) {
+          hasRow = true;
+          scanner->PrintNext(stream, 27);
+        }
+      }
+      stream << "\n";
+    } while (hasRow);
+  }
+}
+
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/file/printer.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/printer.h b/src/parquet/file/printer.h
new file mode 100644
index 0000000..433f9e8
--- /dev/null
+++ b/src/parquet/file/printer.h
@@ -0,0 +1,45 @@
+// 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.
+
+#ifndef PARQUET_FILE_PRINTER_H
+#define PARQUET_FILE_PRINTER_H
+
+#include <cstdint>
+#include <iosfwd>
+#include <list>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/file/reader.h"
+
+namespace parquet {
+
+class PARQUET_EXPORT ParquetFilePrinter {
+ private:
+  ParquetFileReader* fileReader;
+ public:
+  explicit ParquetFilePrinter(ParquetFileReader* reader) : fileReader(reader) {}
+  ~ParquetFilePrinter() {}
+
+  void DebugPrint(
+      std::ostream& stream, std::list<int> selected_columns, bool print_values = true);
+};
+
+}  // namespace parquet
+
+#endif  // PARQUET_FILE_PRINTER_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/file/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader.cc b/src/parquet/file/reader.cc
index 21baecd..7bf2c76 100644
--- a/src/parquet/file/reader.cc
+++ b/src/parquet/file/reader.cc
@@ -22,7 +22,6 @@
 #include <sstream>
 #include <string>
 #include <utility>
-#include <vector>
 
 #include "arrow/io/file.h"
 
@@ -36,7 +35,6 @@
 #include "parquet/util/memory.h"
 
 using std::string;
-using std::vector;
 
 namespace parquet {
 
@@ -127,120 +125,6 @@ std::shared_ptr<RowGroupReader> ParquetFileReader::RowGroup(int i) {
 }
 
 // ----------------------------------------------------------------------
-// ParquetFileReader::DebugPrint
-
-// the fixed initial size is just for an example
-#define COL_WIDTH "30"
-
-void ParquetFileReader::DebugPrint(
-    std::ostream& stream, std::list<int> selected_columns, bool print_values) {
-  const FileMetaData* file_metadata = metadata().get();
-
-  stream << "File statistics:\n";
-  stream << "Version: " << file_metadata->version() << "\n";
-  stream << "Created By: " << file_metadata->created_by() << "\n";
-  stream << "Total rows: " << file_metadata->num_rows() << "\n";
-  stream << "Number of RowGroups: " << file_metadata->num_row_groups() << "\n";
-  stream << "Number of Real Columns: "
-         << file_metadata->schema()->group_node()->field_count() << "\n";
-
-  if (selected_columns.size() == 0) {
-    for (int i = 0; i < file_metadata->num_columns(); i++) {
-      selected_columns.push_back(i);
-    }
-  } else {
-    for (auto i : selected_columns) {
-      if (i < 0 || i >= file_metadata->num_columns()) {
-        throw ParquetException("Selected column is out of range");
-      }
-    }
-  }
-
-  stream << "Number of Columns: " << file_metadata->num_columns() << "\n";
-  stream << "Number of Selected Columns: " << selected_columns.size() << "\n";
-  for (auto i : selected_columns) {
-    const ColumnDescriptor* descr = file_metadata->schema()->Column(i);
-    stream << "Column " << i << ": " << descr->name() << " ("
-           << TypeToString(descr->physical_type()) << ")" << std::endl;
-  }
-
-  for (int r = 0; r < file_metadata->num_row_groups(); ++r) {
-    stream << "--- Row Group " << r << " ---\n";
-
-    auto group_reader = RowGroup(r);
-    std::unique_ptr<RowGroupMetaData> group_metadata = file_metadata->RowGroup(r);
-
-    stream << "--- Total Bytes " << group_metadata->total_byte_size() << " ---\n";
-    stream << "  rows: " << group_metadata->num_rows() << "---\n";
-
-    // Print column metadata
-    for (auto i : selected_columns) {
-      auto column_chunk = group_metadata->ColumnChunk(i);
-      std::shared_ptr<RowGroupStatistics> stats = column_chunk->statistics();
-
-      const ColumnDescriptor* descr = file_metadata->schema()->Column(i);
-      stream << "Column " << i << std::endl << ", values: " << column_chunk->num_values();
-      if (column_chunk->is_stats_set()) {
-        std::string min = stats->EncodeMin(), max = stats->EncodeMax();
-        stream << ", null values: " << stats->null_count()
-               << ", distinct values: " << stats->distinct_count() << std::endl
-               << "  max: " << FormatStatValue(descr->physical_type(), max.c_str())
-               << ", min: " << FormatStatValue(descr->physical_type(), min.c_str());
-      } else {
-        stream << "  Statistics Not Set";
-      }
-      stream << std::endl
-             << "  compression: " << CompressionToString(column_chunk->compression())
-             << ", encodings: ";
-      for (auto encoding : column_chunk->encodings()) {
-        stream << EncodingToString(encoding) << " ";
-      }
-      stream << std::endl
-             << "  uncompressed size: " << column_chunk->total_uncompressed_size()
-             << ", compressed size: " << column_chunk->total_compressed_size()
-             << std::endl;
-    }
-
-    if (!print_values) { continue; }
-
-    static constexpr int bufsize = 25;
-    char buffer[bufsize];
-
-    // Create readers for selected columns and print contents
-    vector<std::shared_ptr<Scanner>> scanners(selected_columns.size(), NULL);
-    int j = 0;
-    for (auto i : selected_columns) {
-      std::shared_ptr<ColumnReader> col_reader = group_reader->Column(i);
-
-      std::stringstream ss;
-      ss << "%-" << COL_WIDTH << "s";
-      std::string fmt = ss.str();
-
-      snprintf(buffer, bufsize, fmt.c_str(),
-          file_metadata->schema()->Column(i)->name().c_str());
-      stream << buffer;
-
-      // This is OK in this method as long as the RowGroupReader does not get
-      // deleted
-      scanners[j++] = Scanner::Make(col_reader);
-    }
-    stream << "\n";
-
-    bool hasRow;
-    do {
-      hasRow = false;
-      for (auto scanner : scanners) {
-        if (scanner->HasNext()) {
-          hasRow = true;
-          scanner->PrintNext(stream, 27);
-        }
-      }
-      stream << "\n";
-    } while (hasRow);
-  }
-}
-
-// ----------------------------------------------------------------------
 // File metadata helpers
 
 std::shared_ptr<FileMetaData> ReadMetaData(

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/file/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader.h b/src/parquet/file/reader.h
index 3cdfa9f..7d3c3f9 100644
--- a/src/parquet/file/reader.h
+++ b/src/parquet/file/reader.h
@@ -111,9 +111,6 @@ class PARQUET_EXPORT ParquetFileReader {
   // Returns the file metadata. Only one instance is ever created
   std::shared_ptr<FileMetaData> metadata() const;
 
-  void DebugPrint(
-      std::ostream& stream, std::list<int> selected_columns, bool print_values = true);
-
  private:
   // Holds a pointer to an instance of Contents implementation
   std::unique_ptr<Contents> contents_;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/src/parquet/reader-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/reader-test.cc b/src/parquet/reader-test.cc
index b0b8851..f7c666c 100644
--- a/src/parquet/reader-test.cc
+++ b/src/parquet/reader-test.cc
@@ -27,6 +27,7 @@
 
 #include "parquet/column/reader.h"
 #include "parquet/column/scanner.h"
+#include "parquet/file/printer.h"
 #include "parquet/file/reader-internal.h"
 #include "parquet/file/reader.h"
 #include "parquet/util/memory.h"
@@ -131,7 +132,8 @@ TEST_F(TestAllTypesPlain, DebugPrintWorks) {
   std::stringstream ss;
 
   std::list<int> columns;
-  reader_->DebugPrint(ss, columns);
+  ParquetFilePrinter printer(reader_.get());
+  printer.DebugPrint(ss, columns);
 
   std::string result = ss.str();
   ASSERT_GT(result.size(), 0);
@@ -144,7 +146,8 @@ TEST_F(TestAllTypesPlain, ColumnSelection) {
   columns.push_back(5);
   columns.push_back(0);
   columns.push_back(10);
-  reader_->DebugPrint(ss, columns);
+  ParquetFilePrinter printer(reader_.get());
+  printer.DebugPrint(ss, columns);
 
   std::string result = ss.str();
   ASSERT_GT(result.size(), 0);
@@ -155,11 +158,13 @@ TEST_F(TestAllTypesPlain, ColumnSelectionOutOfRange) {
 
   std::list<int> columns;
   columns.push_back(100);
-  ASSERT_THROW(reader_->DebugPrint(ss, columns), ParquetException);
+  ParquetFilePrinter printer1(reader_.get());
+  ASSERT_THROW(printer1.DebugPrint(ss, columns), ParquetException);
 
   columns.clear();
   columns.push_back(-1);
-  ASSERT_THROW(reader_->DebugPrint(ss, columns), ParquetException);
+  ParquetFilePrinter printer2(reader_.get());
+  ASSERT_THROW(printer2.DebugPrint(ss, columns), ParquetException);
 }
 
 class TestLocalFile : public ::testing::Test {
@@ -216,7 +221,8 @@ TEST_F(TestLocalFile, OpenWithMetadata) {
   ASSERT_EQ(metadata.get(), reader->metadata().get());
 
   std::list<int> columns;
-  reader->DebugPrint(ss, columns, true);
+  ParquetFilePrinter printer(reader.get());
+  printer.DebugPrint(ss, columns, true);
 
   // Make sure OpenFile passes on the external metadata, too
   auto reader2 = ParquetFileReader::OpenFile(
@@ -237,11 +243,13 @@ TEST(TestFileReaderAdHoc, NationDictTruncatedDataPage) {
 
   // empty list means print all
   std::list<int> columns;
-  reader->DebugPrint(ss, columns, true);
+  ParquetFilePrinter printer1(reader.get());
+  printer1.DebugPrint(ss, columns, true);
 
   reader = ParquetFileReader::OpenFile(nation_dict_truncated_data_page(), true);
   std::stringstream ss2;
-  reader->DebugPrint(ss2, columns, true);
+  ParquetFilePrinter printer2(reader.get());
+  printer2.DebugPrint(ss2, columns, true);
 
   // The memory-mapped reads runs over the end of the column chunk and succeeds
   // by accident

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/a54404ed/tools/parquet_reader.cc
----------------------------------------------------------------------
diff --git a/tools/parquet_reader.cc b/tools/parquet_reader.cc
index bc0711f..25f81c1 100644
--- a/tools/parquet_reader.cc
+++ b/tools/parquet_reader.cc
@@ -57,7 +57,8 @@ int main(int argc, char** argv) {
   try {
     std::unique_ptr<parquet::ParquetFileReader> reader =
         parquet::ParquetFileReader::OpenFile(filename, memory_map);
-    reader->DebugPrint(std::cout, columns, print_values);
+    parquet::ParquetFilePrinter printer(reader.get());
+    printer.DebugPrint(std::cout, columns, print_values);
   } catch (const std::exception& e) {
     std::cerr << "Parquet error: " << e.what() << std::endl;
     return -1;