You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by uw...@apache.org on 2016/12/10 08:06:06 UTC

arrow git commit: ARROW-328: Return shared_ptr by value instead of const-ref

Repository: arrow
Updated Branches:
  refs/heads/master 45ed7e7a3 -> 73fe55683


ARROW-328: Return shared_ptr<T> by value instead of const-ref

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

Closes #235 from wesm/ARROW-328 and squashes the following commits:

f71decc [Wes McKinney] Return shared_ptr<T> by value instead of const-ref


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

Branch: refs/heads/master
Commit: 73fe55683c36465972e21bef01b377c3b66579f9
Parents: 45ed7e7
Author: Wes McKinney <we...@twosigma.com>
Authored: Sat Dec 10 09:05:48 2016 +0100
Committer: Uwe L. Korn <uw...@xhochy.com>
Committed: Sat Dec 10 09:05:48 2016 +0100

----------------------------------------------------------------------
 cpp/src/arrow/array.h                    |  4 ++--
 cpp/src/arrow/builder.h                  |  4 ++--
 cpp/src/arrow/column.cc                  |  2 +-
 cpp/src/arrow/column.h                   |  8 ++++----
 cpp/src/arrow/ipc/file.cc                |  2 +-
 cpp/src/arrow/ipc/file.h                 |  2 +-
 cpp/src/arrow/ipc/metadata-internal.cc   |  2 +-
 cpp/src/arrow/table.h                    |  8 ++++----
 cpp/src/arrow/type.cc                    |  2 +-
 cpp/src/arrow/type.h                     |  6 +++---
 cpp/src/arrow/types/construct.cc         |  2 +-
 cpp/src/arrow/types/list.h               |  6 +++---
 cpp/src/arrow/types/primitive.h          |  2 +-
 cpp/src/arrow/types/struct.h             |  4 ++--
 cpp/src/arrow/util/buffer.h              |  2 +-
 python/pyarrow/includes/libarrow.pxd     | 22 +++++++++++-----------
 python/pyarrow/includes/libarrow_ipc.pxd |  2 +-
 17 files changed, 40 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 78aa2b8..91fb93e 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -53,10 +53,10 @@ class ARROW_EXPORT Array {
   int32_t length() const { return length_; }
   int32_t null_count() const { return null_count_; }
 
-  const std::shared_ptr<DataType>& type() const { return type_; }
+  std::shared_ptr<DataType> type() const { return type_; }
   Type::type type_enum() const { return type_->type; }
 
-  const std::shared_ptr<Buffer>& null_bitmap() const { return null_bitmap_; }
+  std::shared_ptr<Buffer> null_bitmap() const { return null_bitmap_; }
 
   const uint8_t* null_bitmap_data() const { return null_bitmap_data_; }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/builder.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.h b/cpp/src/arrow/builder.h
index cef17e5..73e49c0 100644
--- a/cpp/src/arrow/builder.h
+++ b/cpp/src/arrow/builder.h
@@ -89,13 +89,13 @@ class ARROW_EXPORT ArrayBuilder {
   // this function responsibly.
   Status Advance(int32_t elements);
 
-  const std::shared_ptr<PoolBuffer>& null_bitmap() const { return null_bitmap_; }
+  std::shared_ptr<PoolBuffer> null_bitmap() const { return null_bitmap_; }
 
   // Creates new array object to hold the contents of the builder and transfers
   // ownership of the data.  This resets all variables on the builder.
   virtual Status Finish(std::shared_ptr<Array>* out) = 0;
 
-  const std::shared_ptr<DataType>& type() const { return type_; }
+  std::shared_ptr<DataType> type() const { return type_; }
 
  protected:
   MemoryPool* pool_;

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/column.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/column.cc b/cpp/src/arrow/column.cc
index 52e4c58..eca5f4d 100644
--- a/cpp/src/arrow/column.cc
+++ b/cpp/src/arrow/column.cc
@@ -51,7 +51,7 @@ Column::Column(
 
 Status Column::ValidateData() {
   for (int i = 0; i < data_->num_chunks(); ++i) {
-    const std::shared_ptr<DataType>& type = data_->chunk(i)->type();
+    std::shared_ptr<DataType> type = data_->chunk(i)->type();
     if (!this->type()->Equals(type)) {
       std::stringstream ss;
       ss << "In chunk " << i << " expected type " << this->type()->ToString()

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/column.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/column.h b/cpp/src/arrow/column.h
index d5168cb..1caafec 100644
--- a/cpp/src/arrow/column.h
+++ b/cpp/src/arrow/column.h
@@ -46,7 +46,7 @@ class ARROW_EXPORT ChunkedArray {
 
   int num_chunks() const { return chunks_.size(); }
 
-  const std::shared_ptr<Array>& chunk(int i) const { return chunks_[i]; }
+  std::shared_ptr<Array> chunk(int i) const { return chunks_[i]; }
 
  protected:
   ArrayVector chunks_;
@@ -68,16 +68,16 @@ class ARROW_EXPORT Column {
 
   int64_t null_count() const { return data_->null_count(); }
 
-  const std::shared_ptr<Field>& field() const { return field_; }
+  std::shared_ptr<Field> field() const { return field_; }
 
   // @returns: the column's name in the passed metadata
   const std::string& name() const { return field_->name; }
 
   // @returns: the column's type according to the metadata
-  const std::shared_ptr<DataType>& type() const { return field_->type; }
+  std::shared_ptr<DataType> type() const { return field_->type; }
 
   // @returns: the column's data as a chunked logical array
-  const std::shared_ptr<ChunkedArray>& data() const { return data_; }
+  std::shared_ptr<ChunkedArray> data() const { return data_; }
   // Verify that the column's array data is consistent with the passed field's
   // metadata
   Status ValidateData();

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/ipc/file.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/file.cc b/cpp/src/arrow/ipc/file.cc
index 06001cc..fa50058 100644
--- a/cpp/src/arrow/ipc/file.cc
+++ b/cpp/src/arrow/ipc/file.cc
@@ -179,7 +179,7 @@ Status FileReader::ReadFooter() {
   return footer_->GetSchema(&schema_);
 }
 
-const std::shared_ptr<Schema>& FileReader::schema() const {
+std::shared_ptr<Schema> FileReader::schema() const {
   return schema_;
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/ipc/file.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/file.h b/cpp/src/arrow/ipc/file.h
index 4b79c98..4f35c37 100644
--- a/cpp/src/arrow/ipc/file.h
+++ b/cpp/src/arrow/ipc/file.h
@@ -106,7 +106,7 @@ class ARROW_EXPORT FileReader {
   static Status Open(const std::shared_ptr<io::ReadableFileInterface>& file,
       int64_t footer_offset, std::shared_ptr<FileReader>* reader);
 
-  const std::shared_ptr<Schema>& schema() const;
+  std::shared_ptr<Schema> schema() const;
 
   // Shared dictionaries for dictionary-encoding cross record batches
   // TODO(wesm): Implement dictionary reading when we also have dictionary

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/ipc/metadata-internal.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/metadata-internal.cc b/cpp/src/arrow/ipc/metadata-internal.cc
index 7a24161..5a27589 100644
--- a/cpp/src/arrow/ipc/metadata-internal.cc
+++ b/cpp/src/arrow/ipc/metadata-internal.cc
@@ -285,7 +285,7 @@ Status SchemaToFlatbuffer(
     FBB& fbb, const Schema* schema, flatbuffers::Offset<flatbuf::Schema>* out) {
   std::vector<FieldOffset> field_offsets;
   for (int i = 0; i < schema->num_fields(); ++i) {
-    const std::shared_ptr<Field>& field = schema->field(i);
+    std::shared_ptr<Field> field = schema->field(i);
     FieldOffset offset;
     RETURN_NOT_OK(FieldToFlatbuffer(fbb, field, &offset));
     field_offsets.push_back(offset);

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/table.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index f2c334f..bf5c39f 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -48,11 +48,11 @@ class ARROW_EXPORT RecordBatch {
   bool ApproxEquals(const RecordBatch& other) const;
 
   // @returns: the table's schema
-  const std::shared_ptr<Schema>& schema() const { return schema_; }
+  std::shared_ptr<Schema> schema() const { return schema_; }
 
   // @returns: the i-th column
   // Note: Does not boundscheck
-  const std::shared_ptr<Array>& column(int i) const { return columns_[i]; }
+  std::shared_ptr<Array> column(int i) const { return columns_[i]; }
 
   const std::vector<std::shared_ptr<Array>>& columns() const { return columns_; }
 
@@ -88,11 +88,11 @@ class ARROW_EXPORT Table {
   const std::string& name() const { return name_; }
 
   // @returns: the table's schema
-  const std::shared_ptr<Schema>& schema() const { return schema_; }
+  std::shared_ptr<Schema> schema() const { return schema_; }
 
   // Note: Does not boundscheck
   // @returns: the i-th column
-  const std::shared_ptr<Column>& column(int i) const { return columns_[i]; }
+  std::shared_ptr<Column> column(int i) const { return columns_[i]; }
 
   // @returns: the number of columns in the table
   int num_columns() const { return columns_.size(); }

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index dc955ac..75f5086 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -81,7 +81,7 @@ std::string StructType::ToString() const {
   s << "struct<";
   for (int i = 0; i < this->num_children(); ++i) {
     if (i > 0) { s << ", "; }
-    const std::shared_ptr<Field>& field = this->child(i);
+    std::shared_ptr<Field> field = this->child(i);
     s << field->name << ": " << field->type->ToString();
   }
   s << ">";

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 3077738..966706c 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -134,7 +134,7 @@ struct ARROW_EXPORT DataType {
     return Equals(other.get());
   }
 
-  const std::shared_ptr<Field>& child(int i) const { return children_[i]; }
+  std::shared_ptr<Field> child(int i) const { return children_[i]; }
 
   const std::vector<std::shared_ptr<Field>>& children() const { return children_; }
 
@@ -319,9 +319,9 @@ struct ARROW_EXPORT ListType : public DataType, public NoExtraMeta {
     children_ = {value_field};
   }
 
-  const std::shared_ptr<Field>& value_field() const { return children_[0]; }
+  std::shared_ptr<Field> value_field() const { return children_[0]; }
 
-  const std::shared_ptr<DataType>& value_type() const { return children_[0]->type; }
+  std::shared_ptr<DataType> value_type() const { return children_[0]->type; }
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/types/construct.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/construct.cc b/cpp/src/arrow/types/construct.cc
index 67245f8..ab9c59f 100644
--- a/cpp/src/arrow/types/construct.cc
+++ b/cpp/src/arrow/types/construct.cc
@@ -63,7 +63,7 @@ Status MakeBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type,
 
     case Type::LIST: {
       std::shared_ptr<ArrayBuilder> value_builder;
-      const std::shared_ptr<DataType>& value_type =
+      std::shared_ptr<DataType> value_type =
           static_cast<ListType*>(type.get())->value_type();
       RETURN_NOT_OK(MakeBuilder(pool, value_type, &value_builder));
       out->reset(new ListBuilder(pool, value_builder));

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/types/list.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/list.h b/cpp/src/arrow/types/list.h
index bd93e8f..ec09a78 100644
--- a/cpp/src/arrow/types/list.h
+++ b/cpp/src/arrow/types/list.h
@@ -57,12 +57,12 @@ class ARROW_EXPORT ListArray : public Array {
 
   // Return a shared pointer in case the requestor desires to share ownership
   // with this array.
-  const std::shared_ptr<Array>& values() const { return values_; }
+  std::shared_ptr<Array> values() const { return values_; }
   std::shared_ptr<Buffer> offsets() const {
     return std::static_pointer_cast<Buffer>(offset_buffer_);
   }
 
-  const std::shared_ptr<DataType>& value_type() const { return values_->type(); }
+  std::shared_ptr<DataType> value_type() const { return values_->type(); }
 
   const int32_t* raw_offsets() const { return offsets_; }
 
@@ -152,7 +152,7 @@ class ARROW_EXPORT ListBuilder : public ArrayBuilder {
 
   Status AppendNull() { return Append(false); }
 
-  const std::shared_ptr<ArrayBuilder>& value_builder() const {
+  std::shared_ptr<ArrayBuilder> value_builder() const {
     DCHECK(!values_) << "Using value builder is pointless when values_ is set";
     return value_builder_;
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/types/primitive.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/primitive.h b/cpp/src/arrow/types/primitive.h
index c665218..ec578e1 100644
--- a/cpp/src/arrow/types/primitive.h
+++ b/cpp/src/arrow/types/primitive.h
@@ -45,7 +45,7 @@ class ARROW_EXPORT PrimitiveArray : public Array {
  public:
   virtual ~PrimitiveArray() {}
 
-  const std::shared_ptr<Buffer>& data() const { return data_; }
+  std::shared_ptr<Buffer> data() const { return data_; }
 
   bool EqualsExact(const PrimitiveArray& other) const;
   bool Equals(const std::shared_ptr<Array>& arr) const override;

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/types/struct.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/struct.h b/cpp/src/arrow/types/struct.h
index 035af05..1e2bf2d 100644
--- a/cpp/src/arrow/types/struct.h
+++ b/cpp/src/arrow/types/struct.h
@@ -46,7 +46,7 @@ class ARROW_EXPORT StructArray : public Array {
 
   // Return a shared pointer in case the requestor desires to share ownership
   // with this array.
-  const std::shared_ptr<Array>& field(int32_t pos) const {
+  std::shared_ptr<Array> field(int32_t pos) const {
     DCHECK_GT(field_arrays_.size(), 0);
     return field_arrays_[pos];
   }
@@ -99,7 +99,7 @@ class ARROW_EXPORT StructBuilder : public ArrayBuilder {
 
   Status AppendNull() { return Append(false); }
 
-  const std::shared_ptr<ArrayBuilder> field_builder(int pos) const {
+  std::shared_ptr<ArrayBuilder> field_builder(int pos) const {
     DCHECK_GT(field_builders_.size(), 0);
     return field_builders_[pos];
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/cpp/src/arrow/util/buffer.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/buffer.h b/cpp/src/arrow/util/buffer.h
index 330e15f..5c87395 100644
--- a/cpp/src/arrow/util/buffer.h
+++ b/cpp/src/arrow/util/buffer.h
@@ -86,7 +86,7 @@ class ARROW_EXPORT Buffer : public std::enable_shared_from_this<Buffer> {
 
   int64_t size() const { return size_; }
 
-  const std::shared_ptr<Buffer> parent() const { return parent_; }
+  std::shared_ptr<Buffer> parent() const { return parent_; }
 
  protected:
   bool is_mutable_;

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/python/pyarrow/includes/libarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 350ebe3..15781ce 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -91,12 +91,12 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
         c_bool Equals(const shared_ptr[CSchema]& other)
 
-        const shared_ptr[CField]& field(int i)
+        shared_ptr[CField] field(int i)
         int num_fields()
         c_string ToString()
 
     cdef cppclass CArray" arrow::Array":
-        const shared_ptr[CDataType]& type()
+        shared_ptr[CDataType] type()
 
         int32_t length()
         int32_t null_count()
@@ -142,8 +142,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         const int32_t* offsets()
         int32_t offset(int i)
         int32_t value_length(int i)
-        const shared_ptr[CArray]& values()
-        const shared_ptr[CDataType]& value_type()
+        shared_ptr[CArray] values()
+        shared_ptr[CDataType] value_type()
 
     cdef cppclass CStringArray" arrow::StringArray"(CListArray):
         c_string GetString(int i)
@@ -152,7 +152,7 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         int64_t length()
         int64_t null_count()
         int num_chunks()
-        const shared_ptr[CArray]& chunk(int i)
+        shared_ptr[CArray] chunk(int i)
 
     cdef cppclass CColumn" arrow::Column":
         CColumn(const shared_ptr[CField]& field,
@@ -164,8 +164,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         int64_t length()
         int64_t null_count()
         const c_string& name()
-        const shared_ptr[CDataType]& type()
-        const shared_ptr[CChunkedArray]& data()
+        shared_ptr[CDataType] type()
+        shared_ptr[CChunkedArray] data()
 
     cdef cppclass CRecordBatch" arrow::RecordBatch":
         CRecordBatch(const shared_ptr[CSchema]& schema, int32_t num_rows,
@@ -173,8 +173,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
         c_bool Equals(const CRecordBatch& other)
 
-        const shared_ptr[CSchema]& schema()
-        const shared_ptr[CArray]& column(int i)
+        shared_ptr[CSchema] schema()
+        shared_ptr[CArray] column(int i)
         const c_string& column_name(int i)
 
         const vector[shared_ptr[CArray]]& columns()
@@ -191,8 +191,8 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
 
         const c_string& name()
 
-        const shared_ptr[CSchema]& schema()
-        const shared_ptr[CColumn]& column(int i)
+        shared_ptr[CSchema] schema()
+        shared_ptr[CColumn] column(int i)
 
 
 cdef extern from "arrow/ipc/metadata.h" namespace "arrow::ipc" nogil:

http://git-wip-us.apache.org/repos/asf/arrow/blob/73fe5568/python/pyarrow/includes/libarrow_ipc.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/libarrow_ipc.pxd b/python/pyarrow/includes/libarrow_ipc.pxd
index eda5b9b..b3185b1 100644
--- a/python/pyarrow/includes/libarrow_ipc.pxd
+++ b/python/pyarrow/includes/libarrow_ipc.pxd
@@ -44,7 +44,7 @@ cdef extern from "arrow/ipc/file.h" namespace "arrow::ipc" nogil:
         CStatus Open2" Open"(const shared_ptr[ReadableFileInterface]& file,
                      int64_t footer_offset, shared_ptr[CFileReader]* out)
 
-        const shared_ptr[CSchema]& schema()
+        shared_ptr[CSchema] schema()
 
         int num_dictionaries()
         int num_record_batches()