You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/09/06 11:11:48 UTC

arrow git commit: ARROW-1396: [C++] Add PrettyPrint for schemas that outputs dictionaries

Repository: arrow
Updated Branches:
  refs/heads/master e9f3a12da -> a3514a388


ARROW-1396: [C++] Add PrettyPrint for schemas that outputs dictionaries

Author: Wes McKinney <we...@twosigma.com>

Closes #1051 from wesm/ARROW-1396 and squashes the following commits:

11a8b690 [Wes McKinney] Fix cpplint error
81637eae [Wes McKinney] clang-format
dfd46924 [Wes McKinney] Use PrettyPrint with schemas in Python
43652bce [Wes McKinney] Add PrettyPrint functions for Schema that also print dictionaries
f097675e [Wes McKinney] First draft of PrettyPrint for schemas, untested. Name DataType::name a virtual


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

Branch: refs/heads/master
Commit: a3514a3889224125bb115a67ff71ab57fd01cfa0
Parents: e9f3a12
Author: Wes McKinney <we...@twosigma.com>
Authored: Wed Sep 6 07:11:43 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Wed Sep 6 07:11:43 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/pretty_print-test.cc   |  40 +++++++-
 cpp/src/arrow/pretty_print.cc        | 148 +++++++++++++++++++++++-------
 cpp/src/arrow/pretty_print.h         |  21 ++++-
 cpp/src/arrow/type.h                 |  79 +++++++++-------
 python/pyarrow/includes/libarrow.pxd |   7 ++
 python/pyarrow/tests/test_schema.py  |  20 +++-
 python/pyarrow/types.pxi             |  14 ++-
 7 files changed, 258 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/cpp/src/arrow/pretty_print-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print-test.cc b/cpp/src/arrow/pretty_print-test.cc
index a687a8f..d4c92ce 100644
--- a/cpp/src/arrow/pretty_print-test.cc
+++ b/cpp/src/arrow/pretty_print-test.cc
@@ -55,6 +55,13 @@ void CheckArray(const Array& arr, int indent, const char* expected) {
   ASSERT_EQ(result, ss.str());
 }
 
+template <typename T>
+void Check(const T& obj, const PrettyPrintOptions& options, const char* expected) {
+  std::string result;
+  ASSERT_OK(PrettyPrint(obj, options, &result));
+  ASSERT_EQ(std::string(expected, strlen(expected)), result);
+}
+
 template <typename TYPE, typename C_TYPE>
 void CheckPrimitive(int indent, const std::vector<bool>& is_valid,
                     const std::vector<C_TYPE>& values, const char* expected) {
@@ -117,7 +124,38 @@ TEST_F(TestPrettyPrint, DictionaryType) {
 -- dictionary: ["foo", "bar", "baz"]
 -- indices: [1, 2, null, 0, 2, 0])expected";
 
-  CheckArray(*arr.get(), 0, expected);
+  CheckArray(*arr, 0, expected);
+}
+
+TEST_F(TestPrettyPrint, SchemaWithDictionary) {
+  std::vector<bool> is_valid = {true, true, false, true, true, true};
+
+  std::shared_ptr<Array> dict;
+  std::vector<std::string> dict_values = {"foo", "bar", "baz"};
+  ArrayFromVector<StringType, std::string>(dict_values, &dict);
+
+  auto simple = field("one", int32());
+  auto simple_dict = field("two", dictionary(int16(), dict));
+  auto list_of_dict = field("three", list(simple_dict));
+
+  auto struct_with_dict = field("four", struct_({simple, simple_dict}));
+
+  auto sch = schema({simple, simple_dict, list_of_dict, struct_with_dict});
+
+  static const char* expected = R"expected(one: int32
+two: dictionary<values=string, indices=int16, ordered=0>
+  dictionary: ["foo", "bar", "baz"]
+three: list<two: dictionary<values=string, indices=int16, ordered=0>>
+  child 0, two: dictionary<values=string, indices=int16, ordered=0>
+      dictionary: ["foo", "bar", "baz"]
+four: struct<one: int32, two: dictionary<values=string, indices=int16, ordered=0>>
+  child 0, one: int32
+  child 1, two: dictionary<values=string, indices=int16, ordered=0>
+      dictionary: ["foo", "bar", "baz"])expected";
+
+  PrettyPrintOptions options{0};
+
+  Check(*sch, options, expected);
 }
 
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/cpp/src/arrow/pretty_print.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.cc b/cpp/src/arrow/pretty_print.cc
index 7794f8d..f759056 100644
--- a/cpp/src/arrow/pretty_print.cc
+++ b/cpp/src/arrow/pretty_print.cc
@@ -32,10 +32,56 @@
 
 namespace arrow {
 
-class ArrayPrinter {
+class PrettyPrinter {
+ public:
+  PrettyPrinter(int indent, std::ostream* sink) : indent_(indent), sink_(sink) {}
+
+  void Write(const char* data);
+  void Write(const std::string& data);
+  void WriteIndented(const char* data);
+  void WriteIndented(const std::string& data);
+  void Newline();
+  void Indent();
+  void OpenArray();
+  void CloseArray();
+
+ protected:
+  int indent_;
+  std::ostream* sink_;
+};
+
+void PrettyPrinter::OpenArray() { (*sink_) << "["; }
+
+void PrettyPrinter::CloseArray() { (*sink_) << "]"; }
+
+void PrettyPrinter::Write(const char* data) { (*sink_) << data; }
+void PrettyPrinter::Write(const std::string& data) { (*sink_) << data; }
+
+void PrettyPrinter::WriteIndented(const char* data) {
+  Indent();
+  Write(data);
+}
+
+void PrettyPrinter::WriteIndented(const std::string& data) {
+  Indent();
+  Write(data);
+}
+
+void PrettyPrinter::Newline() {
+  (*sink_) << "\n";
+  Indent();
+}
+
+void PrettyPrinter::Indent() {
+  for (int i = 0; i < indent_; ++i) {
+    (*sink_) << " ";
+  }
+}
+
+class ArrayPrinter : public PrettyPrinter {
  public:
   ArrayPrinter(const Array& array, int indent, std::ostream* sink)
-      : array_(array), indent_(indent), sink_(sink) {}
+      : PrettyPrinter(indent, sink), array_(array) {}
 
   template <typename T>
   inline typename std::enable_if<IsInteger<T>::value, void>::type WriteDataValues(
@@ -136,13 +182,6 @@ class ArrayPrinter {
     }
   }
 
-  void Write(const char* data);
-  void Write(const std::string& data);
-  void Newline();
-  void Indent();
-  void OpenArray();
-  void CloseArray();
-
   Status Visit(const NullArray& array) { return Status::OK(); }
 
   template <typename T>
@@ -250,9 +289,6 @@ class ArrayPrinter {
 
  private:
   const Array& array_;
-  int indent_;
-
-  std::ostream* sink_;
 };
 
 Status ArrayPrinter::WriteValidityBitmap(const Array& array) {
@@ -269,24 +305,6 @@ Status ArrayPrinter::WriteValidityBitmap(const Array& array) {
   }
 }
 
-void ArrayPrinter::OpenArray() { (*sink_) << "["; }
-void ArrayPrinter::CloseArray() { (*sink_) << "]"; }
-
-void ArrayPrinter::Write(const char* data) { (*sink_) << data; }
-
-void ArrayPrinter::Write(const std::string& data) { (*sink_) << data; }
-
-void ArrayPrinter::Newline() {
-  (*sink_) << "\n";
-  Indent();
-}
-
-void ArrayPrinter::Indent() {
-  for (int i = 0; i < indent_; ++i) {
-    (*sink_) << " ";
-  }
-}
-
 Status PrettyPrint(const Array& arr, int indent, std::ostream* sink) {
   ArrayPrinter printer(arr, indent, sink);
   return printer.Print();
@@ -302,8 +320,76 @@ Status PrettyPrint(const RecordBatch& batch, int indent, std::ostream* sink) {
   return Status::OK();
 }
 
-Status ARROW_EXPORT DebugPrint(const Array& arr, int indent) {
+Status DebugPrint(const Array& arr, int indent) {
   return PrettyPrint(arr, indent, &std::cout);
 }
 
+class SchemaPrinter : public PrettyPrinter {
+ public:
+  SchemaPrinter(const Schema& schema, int indent, std::ostream* sink)
+      : PrettyPrinter(indent, sink), schema_(schema) {}
+
+  Status PrintType(const DataType& type);
+  Status PrintField(const Field& field);
+
+  Status Print() {
+    for (int i = 0; i < schema_.num_fields(); ++i) {
+      if (i > 0) {
+        Newline();
+      }
+      RETURN_NOT_OK(PrintField(*schema_.field(i)));
+    }
+    return Status::OK();
+  }
+
+ private:
+  const Schema& schema_;
+};
+
+Status SchemaPrinter::PrintType(const DataType& type) {
+  Write(type.ToString());
+  if (type.id() == Type::DICTIONARY) {
+    Newline();
+
+    indent_ += 2;
+    WriteIndented("dictionary: ");
+    const auto& dict_type = static_cast<const DictionaryType&>(type);
+    RETURN_NOT_OK(PrettyPrint(*dict_type.dictionary(), indent_, sink_));
+    indent_ -= 2;
+  } else {
+    for (int i = 0; i < type.num_children(); ++i) {
+      Newline();
+
+      std::stringstream ss;
+      ss << "child " << i << ", ";
+
+      indent_ += 2;
+      WriteIndented(ss.str());
+      RETURN_NOT_OK(PrintField(*type.child(i)));
+      indent_ -= 2;
+    }
+  }
+  return Status::OK();
+}
+
+Status SchemaPrinter::PrintField(const Field& field) {
+  Write(field.name());
+  Write(": ");
+  return PrintType(*field.type());
+}
+
+Status PrettyPrint(const Schema& schema, const PrettyPrintOptions& options,
+                   std::ostream* sink) {
+  SchemaPrinter printer(schema, options.indent, sink);
+  return printer.Print();
+}
+
+Status PrettyPrint(const Schema& schema, const PrettyPrintOptions& options,
+                   std::string* result) {
+  std::ostringstream sink;
+  RETURN_NOT_OK(PrettyPrint(schema, options, &sink));
+  *result = sink.str();
+  return Status::OK();
+}
+
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/cpp/src/arrow/pretty_print.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.h b/cpp/src/arrow/pretty_print.h
index a45c8a8..e3aa020 100644
--- a/cpp/src/arrow/pretty_print.h
+++ b/cpp/src/arrow/pretty_print.h
@@ -19,6 +19,7 @@
 #define ARROW_PRETTY_PRINT_H
 
 #include <ostream>
+#include <string>
 
 #include "arrow/type_fwd.h"
 #include "arrow/util/visibility.h"
@@ -32,10 +33,24 @@ struct PrettyPrintOptions {
   int indent;
 };
 
-Status ARROW_EXPORT PrettyPrint(const RecordBatch& batch, int indent, std::ostream* sink);
-Status ARROW_EXPORT PrettyPrint(const Array& arr, int indent, std::ostream* sink);
+/// \brief Print human-readable representation of RecordBatch
+ARROW_EXPORT
+Status PrettyPrint(const RecordBatch& batch, int indent, std::ostream* sink);
 
-Status ARROW_EXPORT DebugPrint(const Array& arr, int indent);
+/// \brief Print human-readable representation of Array
+ARROW_EXPORT
+Status PrettyPrint(const Array& arr, int indent, std::ostream* sink);
+
+ARROW_EXPORT
+Status PrettyPrint(const Schema& schema, const PrettyPrintOptions& options,
+                   std::ostream* sink);
+
+ARROW_EXPORT
+Status PrettyPrint(const Schema& schema, const PrettyPrintOptions& options,
+                   std::string* result);
+
+ARROW_EXPORT
+Status DebugPrint(const Array& arr, int indent);
 
 }  // namespace arrow
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index d197817..b532cd2 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -148,8 +148,15 @@ class ARROW_EXPORT DataType {
 
   virtual Status Accept(TypeVisitor* visitor) const = 0;
 
+  /// \brief A string representation of the type, including any children
   virtual std::string ToString() const = 0;
 
+  /// \brief A string name of the type, omitting any child fields
+  ///
+  /// \note Experimental API
+  /// \since 0.7.0
+  virtual std::string name() const = 0;
+
   virtual std::vector<BufferDescr> GetBufferLayout() const = 0;
 
   Type::type id() const { return id_; }
@@ -195,7 +202,6 @@ class ARROW_EXPORT FloatingPoint : public PrimitiveCType {
 class ARROW_EXPORT NestedType : public DataType {
  public:
   using DataType::DataType;
-  static std::string name() { return "nested"; }
 };
 
 class NoExtraMeta {};
@@ -258,7 +264,7 @@ class ARROW_EXPORT CTypeImpl : public BASE {
     return visitor->Visit(*static_cast<const DERIVED*>(this));
   }
 
-  std::string ToString() const override { return std::string(DERIVED::name()); }
+  std::string ToString() const override { return this->name(); }
 };
 
 template <typename DERIVED, Type::type TYPE_ID, typename C_TYPE>
@@ -277,7 +283,7 @@ class ARROW_EXPORT NullType : public DataType, public NoExtraMeta {
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
 
-  static std::string name() { return "null"; }
+  std::string name() const override { return "null"; }
 
   std::vector<BufferDescr> GetBufferLayout() const override;
 };
@@ -292,76 +298,76 @@ class ARROW_EXPORT BooleanType : public FixedWidthType, public NoExtraMeta {
   std::string ToString() const override;
 
   int bit_width() const override { return 1; }
-  static std::string name() { return "bool"; }
+  std::string name() const override { return "bool"; }
 };
 
 class ARROW_EXPORT UInt8Type
     : public detail::IntegerTypeImpl<UInt8Type, Type::UINT8, uint8_t> {
  public:
-  static std::string name() { return "uint8"; }
+  std::string name() const override { return "uint8"; }
 };
 
 class ARROW_EXPORT Int8Type
     : public detail::IntegerTypeImpl<Int8Type, Type::INT8, int8_t> {
  public:
-  static std::string name() { return "int8"; }
+  std::string name() const override { return "int8"; }
 };
 
 class ARROW_EXPORT UInt16Type
     : public detail::IntegerTypeImpl<UInt16Type, Type::UINT16, uint16_t> {
  public:
-  static std::string name() { return "uint16"; }
+  std::string name() const override { return "uint16"; }
 };
 
 class ARROW_EXPORT Int16Type
     : public detail::IntegerTypeImpl<Int16Type, Type::INT16, int16_t> {
  public:
-  static std::string name() { return "int16"; }
+  std::string name() const override { return "int16"; }
 };
 
 class ARROW_EXPORT UInt32Type
     : public detail::IntegerTypeImpl<UInt32Type, Type::UINT32, uint32_t> {
  public:
-  static std::string name() { return "uint32"; }
+  std::string name() const override { return "uint32"; }
 };
 
 class ARROW_EXPORT Int32Type
     : public detail::IntegerTypeImpl<Int32Type, Type::INT32, int32_t> {
  public:
-  static std::string name() { return "int32"; }
+  std::string name() const override { return "int32"; }
 };
 
 class ARROW_EXPORT UInt64Type
     : public detail::IntegerTypeImpl<UInt64Type, Type::UINT64, uint64_t> {
  public:
-  static std::string name() { return "uint64"; }
+  std::string name() const override { return "uint64"; }
 };
 
 class ARROW_EXPORT Int64Type
     : public detail::IntegerTypeImpl<Int64Type, Type::INT64, int64_t> {
  public:
-  static std::string name() { return "int64"; }
+  std::string name() const override { return "int64"; }
 };
 
 class ARROW_EXPORT HalfFloatType
     : public detail::CTypeImpl<HalfFloatType, FloatingPoint, Type::HALF_FLOAT, uint16_t> {
  public:
   Precision precision() const override;
-  static std::string name() { return "halffloat"; }
+  std::string name() const override { return "halffloat"; }
 };
 
 class ARROW_EXPORT FloatType
     : public detail::CTypeImpl<FloatType, FloatingPoint, Type::FLOAT, float> {
  public:
   Precision precision() const override;
-  static std::string name() { return "float"; }
+  std::string name() const override { return "float"; }
 };
 
 class ARROW_EXPORT DoubleType
     : public detail::CTypeImpl<DoubleType, FloatingPoint, Type::DOUBLE, double> {
  public:
   Precision precision() const override;
-  static std::string name() { return "double"; }
+  std::string name() const override { return "double"; }
 };
 
 class ARROW_EXPORT ListType : public NestedType {
@@ -383,7 +389,7 @@ class ARROW_EXPORT ListType : public NestedType {
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
 
-  static std::string name() { return "list"; }
+  std::string name() const override { return "list"; }
 
   std::vector<BufferDescr> GetBufferLayout() const override;
 };
@@ -397,7 +403,7 @@ class ARROW_EXPORT BinaryType : public DataType, public NoExtraMeta {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "binary"; }
+  std::string name() const override { return "binary"; }
 
   std::vector<BufferDescr> GetBufferLayout() const override;
 
@@ -418,7 +424,7 @@ class ARROW_EXPORT FixedSizeBinaryType : public FixedWidthType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "fixed_size_binary"; }
+  std::string name() const override { return "fixed_size_binary"; }
 
   std::vector<BufferDescr> GetBufferLayout() const override;
 
@@ -438,7 +444,7 @@ class ARROW_EXPORT StringType : public BinaryType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "utf8"; }
+  std::string name() const override { return "utf8"; }
 };
 
 class ARROW_EXPORT StructType : public NestedType {
@@ -452,7 +458,7 @@ class ARROW_EXPORT StructType : public NestedType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "struct"; }
+  std::string name() const override { return "struct"; }
 
   std::vector<BufferDescr> GetBufferLayout() const override;
 };
@@ -466,7 +472,7 @@ class ARROW_EXPORT DecimalType : public FixedSizeBinaryType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "decimal"; }
+  std::string name() const override { return "decimal"; }
 
   int32_t precision() const { return precision_; }
   int32_t scale() const { return scale_; }
@@ -486,7 +492,7 @@ class ARROW_EXPORT UnionType : public NestedType {
             const std::vector<uint8_t>& type_codes, UnionMode mode = UnionMode::SPARSE);
 
   std::string ToString() const override;
-  static std::string name() { return "union"; }
+  std::string name() const override { return "union"; }
   Status Accept(TypeVisitor* visitor) const override;
 
   std::vector<BufferDescr> GetBufferLayout() const override;
@@ -531,6 +537,8 @@ class ARROW_EXPORT Date32Type : public DateType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
+
+  std::string name() const override { return "date32"; }
 };
 
 /// Date as int64_t milliseconds since UNIX epoch
@@ -546,7 +554,8 @@ class ARROW_EXPORT Date64Type : public DateType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "date"; }
+
+  std::string name() const override { return "date64"; }
 };
 
 struct TimeUnit {
@@ -591,6 +600,8 @@ class ARROW_EXPORT Time32Type : public TimeType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
+
+  std::string name() const override { return "time32"; }
 };
 
 class ARROW_EXPORT Time64Type : public TimeType {
@@ -604,6 +615,8 @@ class ARROW_EXPORT Time64Type : public TimeType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
+
+  std::string name() const override { return "time64"; }
 };
 
 class ARROW_EXPORT TimestampType : public FixedWidthType {
@@ -623,7 +636,7 @@ class ARROW_EXPORT TimestampType : public FixedWidthType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "timestamp"; }
+  std::string name() const override { return "timestamp"; }
 
   TimeUnit::type unit() const { return unit_; }
   const std::string& timezone() const { return timezone_; }
@@ -647,7 +660,7 @@ class ARROW_EXPORT IntervalType : public FixedWidthType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override { return name(); }
-  static std::string name() { return "date"; }
+  std::string name() const override { return "date"; }
 
   Unit unit() const { return unit_; }
 
@@ -673,7 +686,7 @@ class ARROW_EXPORT DictionaryType : public FixedWidthType {
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
-  static std::string name() { return "dictionary"; }
+  std::string name() const override { return "dictionary"; }
 
   bool ordered() const { return ordered_; }
 
@@ -814,18 +827,20 @@ std::shared_ptr<Field> ARROW_EXPORT field(
 /// \param fields the schema's fields
 /// \param metadata any custom key-value metadata, default nullptr
 /// \return schema shared_ptr to Schema
-std::shared_ptr<Schema> ARROW_EXPORT
-schema(const std::vector<std::shared_ptr<Field>>& fields,
-       const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
+ARROW_EXPORT
+std::shared_ptr<Schema> schema(
+    const std::vector<std::shared_ptr<Field>>& fields,
+    const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
 
 /// \brief Create a Schema instance
 ///
 /// \param fields the schema's fields (rvalue reference)
 /// \param metadata any custom key-value metadata, default nullptr
 /// \return schema shared_ptr to Schema
-std::shared_ptr<Schema> ARROW_EXPORT
-schema(std::vector<std::shared_ptr<Field>>&& fields,
-       const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
+ARROW_EXPORT
+std::shared_ptr<Schema> schema(
+    std::vector<std::shared_ptr<Field>>&& fields,
+    const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
 
 // ----------------------------------------------------------------------
 //

http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/python/pyarrow/includes/libarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index e032448..6bb8702 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -242,6 +242,13 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
             const shared_ptr[CKeyValueMetadata]& metadata)
         shared_ptr[CSchema] RemoveMetadata()
 
+    cdef cppclass PrettyPrintOptions:
+        int indent;
+
+    CStatus PrettyPrint(const CSchema& schema,
+                        const PrettyPrintOptions& options,
+                        c_string* result)
+
     cdef cppclass CBooleanArray" arrow::BooleanArray"(CArray):
         c_bool Value(int i)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/python/pyarrow/tests/test_schema.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_schema.py b/python/pyarrow/tests/test_schema.py
index f920e8d..4bb6a5a 100644
--- a/python/pyarrow/tests/test_schema.py
+++ b/python/pyarrow/tests/test_schema.py
@@ -147,7 +147,8 @@ def test_schema():
     assert repr(sch) == """\
 foo: int32
 bar: string
-baz: list<item: int8>"""
+baz: list<item: int8>
+  child 0, item: int8"""
 
 
 def test_field_empty():
@@ -236,3 +237,20 @@ def test_schema_negative_indexing():
 
     with pytest.raises(IndexError):
         schema[3]
+
+
+def test_schema_repr_with_dictionaries():
+    dct = pa.array(['foo', 'bar', 'baz'], type=pa.string())
+    fields = [
+        pa.field('one', pa.dictionary(pa.int16(), dct)),
+        pa.field('two', pa.int32())
+    ]
+    sch = pa.schema(fields)
+
+    expected = (
+        """\
+one: dictionary<values=string, indices=int16, ordered=0>
+  dictionary: ["foo", "bar", "baz"]
+two: int32""")
+
+    assert repr(sch) == expected

http://git-wip-us.apache.org/repos/asf/arrow/blob/a3514a38/python/pyarrow/types.pxi
----------------------------------------------------------------------
diff --git a/python/pyarrow/types.pxi b/python/pyarrow/types.pxi
index 30c3aa6..fb6b961 100644
--- a/python/pyarrow/types.pxi
+++ b/python/pyarrow/types.pxi
@@ -423,7 +423,15 @@ cdef class Schema:
         return pyarrow_wrap_schema(new_schema)
 
     def __str__(self):
-        return frombytes(self.schema.ToString())
+        cdef:
+            PrettyPrintOptions options
+            c_string result
+
+        options.indent = 0
+        with nogil:
+            check_status(PrettyPrint(deref(self.schema), options, &result))
+
+        return frombytes(result)
 
     def __repr__(self):
         return self.__str__()
@@ -835,7 +843,7 @@ cpdef ListType list_(value_type):
     return out
 
 
-cpdef DictionaryType dictionary(DataType index_type, Array dictionary,
+cpdef DictionaryType dictionary(DataType index_type, Array dict_values,
                                 bint ordered=False):
     """
     Dictionary (categorical, or simply encoded) type
@@ -852,7 +860,7 @@ cpdef DictionaryType dictionary(DataType index_type, Array dictionary,
     cdef DictionaryType out = DictionaryType()
     cdef shared_ptr[CDataType] dict_type
     dict_type.reset(new CDictionaryType(index_type.sp_type,
-                                        dictionary.sp_array,
+                                        dict_values.sp_array,
                                         ordered == 1))
     out.init(dict_type)
     return out