Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id EAD201660B5 for ; Tue, 25 Jul 2017 04:55:59 +0200 (CEST) Received: (qmail 94972 invoked by uid 500); 25 Jul 2017 02:55:58 -0000 Mailing-List: contact commits-help@arrow.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@arrow.apache.org Delivered-To: mailing list commits@arrow.apache.org Received: (qmail 94797 invoked by uid 99); 25 Jul 2017 02:55:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 25 Jul 2017 02:55:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id A614CF2178; Tue, 25 Jul 2017 02:55:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wesm@apache.org To: commits@arrow.apache.org Date: Tue, 25 Jul 2017 02:56:01 -0000 Message-Id: In-Reply-To: <4c4b3befa64343adbee8422db8c968e7@git.apache.org> References: <4c4b3befa64343adbee8422db8c968e7@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [5/7] arrow git commit: ARROW-1219: [C++] Use Google C++ code formatting http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/interfaces.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/interfaces.cc b/cpp/src/arrow/io/interfaces.cc index 06957d4..57dc42d 100644 --- a/cpp/src/arrow/io/interfaces.cc +++ b/cpp/src/arrow/io/interfaces.cc @@ -29,32 +29,28 @@ namespace io { FileInterface::~FileInterface() {} -RandomAccessFile::RandomAccessFile() { - set_mode(FileMode::READ); -} +RandomAccessFile::RandomAccessFile() { set_mode(FileMode::READ); } -Status RandomAccessFile::ReadAt( - int64_t position, int64_t nbytes, int64_t* bytes_read, uint8_t* out) { +Status RandomAccessFile::ReadAt(int64_t position, int64_t nbytes, int64_t* bytes_read, + uint8_t* out) { std::lock_guard guard(lock_); RETURN_NOT_OK(Seek(position)); return Read(nbytes, bytes_read, out); } -Status RandomAccessFile::ReadAt( - int64_t position, int64_t nbytes, std::shared_ptr* out) { +Status RandomAccessFile::ReadAt(int64_t position, int64_t nbytes, + std::shared_ptr* out) { std::lock_guard guard(lock_); RETURN_NOT_OK(Seek(position)); return Read(nbytes, out); } Status Writeable::Write(const std::string& data) { - return Write( - reinterpret_cast(data.c_str()), static_cast(data.size())); + return Write(reinterpret_cast(data.c_str()), + static_cast(data.size())); } -Status Writeable::Flush() { - return Status::OK(); -} +Status Writeable::Flush() { return Status::OK(); } } // namespace io } // namespace arrow http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/interfaces.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/interfaces.h b/cpp/src/arrow/io/interfaces.h index b5a0bd8..e71a5c9 100644 --- a/cpp/src/arrow/io/interfaces.h +++ b/cpp/src/arrow/io/interfaces.h @@ -107,8 +107,8 @@ class ARROW_EXPORT RandomAccessFile : public InputStream, public Seekable { /// be overridden /// /// Default implementation is thread-safe - virtual Status ReadAt( - int64_t position, int64_t nbytes, int64_t* bytes_read, uint8_t* out); + virtual Status ReadAt(int64_t position, int64_t nbytes, int64_t* bytes_read, + uint8_t* out); /// Default implementation is thread-safe virtual Status ReadAt(int64_t position, int64_t nbytes, std::shared_ptr* out); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/io-file-test.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/io-file-test.cc b/cpp/src/arrow/io/io-file-test.cc index a077f8c..36c3570 100644 --- a/cpp/src/arrow/io/io-file-test.cc +++ b/cpp/src/arrow/io/io-file-test.cc @@ -43,9 +43,10 @@ static bool FileExists(const std::string& path) { #if defined(_MSC_VER) void InvalidParamHandler(const wchar_t* expr, const wchar_t* func, - const wchar_t* source_file, unsigned int source_line, uintptr_t reserved) { + const wchar_t* source_file, unsigned int source_line, + uintptr_t reserved) { wprintf(L"Invalid parameter in funcion %s. Source: %s line %d expression %s", func, - source_file, source_line, expr); + source_file, source_line, expr); } #endif @@ -61,7 +62,9 @@ static bool FileIsClosed(int fd) { int ret = static_cast(_close(fd)); return (ret == -1); #else - if (-1 != fcntl(fd, F_GETFD)) { return false; } + if (-1 != fcntl(fd, F_GETFD)) { + return false; + } return errno == EBADF; #endif } @@ -76,7 +79,9 @@ class FileTestFixture : public ::testing::Test { void TearDown() { EnsureFileDeleted(); } void EnsureFileDeleted() { - if (FileExists(path_)) { std::remove(path_.c_str()); } + if (FileExists(path_)) { + std::remove(path_.c_str()); + } } protected: @@ -382,7 +387,9 @@ TEST_F(TestReadableFile, ThreadSafety) { for (int i = 0; i < niter; ++i) { ASSERT_OK(file_->ReadAt(0, 3, &buffer)); - if (0 == memcmp(data.c_str(), buffer->data(), 3)) { correct_count += 1; } + if (0 == memcmp(data.c_str(), buffer->data(), 3)) { + correct_count += 1; + } } }; @@ -547,8 +554,8 @@ TEST_F(TestMemoryMappedFile, InvalidFile) { std::string non_existent_path = "invalid-file-name-asfd"; std::shared_ptr result; - ASSERT_RAISES( - IOError, MemoryMappedFile::Open(non_existent_path, FileMode::READ, &result)); + ASSERT_RAISES(IOError, + MemoryMappedFile::Open(non_existent_path, FileMode::READ, &result)); } TEST_F(TestMemoryMappedFile, CastableToFileInterface) { @@ -563,8 +570,8 @@ TEST_F(TestMemoryMappedFile, ThreadSafety) { std::shared_ptr file; ASSERT_OK(MemoryMappedFile::Open(path, FileMode::READWRITE, &file)); - ASSERT_OK(file->Write( - reinterpret_cast(data.c_str()), static_cast(data.size()))); + ASSERT_OK(file->Write(reinterpret_cast(data.c_str()), + static_cast(data.size()))); std::atomic correct_count(0); const int niter = 10000; @@ -574,7 +581,9 @@ TEST_F(TestMemoryMappedFile, ThreadSafety) { for (int i = 0; i < niter; ++i) { ASSERT_OK(file->ReadAt(0, 3, &buffer)); - if (0 == memcmp(data.c_str(), buffer->data(), 3)) { correct_count += 1; } + if (0 == memcmp(data.c_str(), buffer->data(), 3)) { + correct_count += 1; + } } }; http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/io-hdfs-test.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/io-hdfs-test.cc b/cpp/src/arrow/io/io-hdfs-test.cc index 74f8042..c584cf5 100644 --- a/cpp/src/arrow/io/io-hdfs-test.cc +++ b/cpp/src/arrow/io/io-hdfs-test.cc @@ -58,11 +58,11 @@ class TestHdfsClient : public ::testing::Test { } Status WriteDummyFile(const std::string& path, const uint8_t* buffer, int64_t size, - bool append = false, int buffer_size = 0, int16_t replication = 0, - int default_block_size = 0) { + bool append = false, int buffer_size = 0, int16_t replication = 0, + int default_block_size = 0) { std::shared_ptr file; - RETURN_NOT_OK(client_->OpenWriteable( - path, append, buffer_size, replication, default_block_size, &file)); + RETURN_NOT_OK(client_->OpenWriteable(path, append, buffer_size, replication, + default_block_size, &file)); RETURN_NOT_OK(file->Write(buffer, size)); RETURN_NOT_OK(file->Close()); @@ -87,9 +87,10 @@ class TestHdfsClient : public ::testing::Test { LibHdfsShim* driver_shim; client_ = nullptr; - scratch_dir_ = boost::filesystem::unique_path( - boost::filesystem::temp_directory_path() / "arrow-hdfs/scratch-%%%%") - .string(); + scratch_dir_ = + boost::filesystem::unique_path(boost::filesystem::temp_directory_path() / + "arrow-hdfs/scratch-%%%%") + .string(); loaded_driver_ = false; @@ -175,7 +176,9 @@ TYPED_TEST(TestHdfsClient, MakeDirectory) { std::string path = this->ScratchPath("create-directory"); - if (this->client_->Exists(path)) { ASSERT_OK(this->client_->Delete(path, true)); } + if (this->client_->Exists(path)) { + ASSERT_OK(this->client_->Delete(path, true)); + } ASSERT_OK(this->client_->MakeDirectory(path)); ASSERT_TRUE(this->client_->Exists(path)); @@ -396,7 +399,7 @@ TYPED_TEST(TestHdfsClient, ThreadSafety) { std::string data = "foobar"; ASSERT_OK(this->WriteDummyFile(src_path, reinterpret_cast(data.c_str()), - static_cast(data.size()))); + static_cast(data.size()))); std::shared_ptr file; ASSERT_OK(this->client_->OpenReadable(src_path, &file)); @@ -409,10 +412,14 @@ TYPED_TEST(TestHdfsClient, ThreadSafety) { std::shared_ptr buffer; if (i % 2 == 0) { ASSERT_OK(file->ReadAt(3, 3, &buffer)); - if (0 == memcmp(data.c_str() + 3, buffer->data(), 3)) { correct_count += 1; } + if (0 == memcmp(data.c_str() + 3, buffer->data(), 3)) { + correct_count += 1; + } } else { ASSERT_OK(file->ReadAt(0, 4, &buffer)); - if (0 == memcmp(data.c_str() + 0, buffer->data(), 4)) { correct_count += 1; } + if (0 == memcmp(data.c_str() + 0, buffer->data(), 4)) { + correct_count += 1; + } } } }; http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/memory.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/memory.cc b/cpp/src/arrow/io/memory.cc index 4d8bf63..b6c48ec 100644 --- a/cpp/src/arrow/io/memory.cc +++ b/cpp/src/arrow/io/memory.cc @@ -46,7 +46,7 @@ BufferOutputStream::BufferOutputStream(const std::shared_ptr& b mutable_data_(buffer->mutable_data()) {} Status BufferOutputStream::Create(int64_t initial_capacity, MemoryPool* pool, - std::shared_ptr* out) { + std::shared_ptr* out) { std::shared_ptr buffer; RETURN_NOT_OK(AllocateResizableBuffer(pool, initial_capacity, &buffer)); *out = std::make_shared(buffer); @@ -55,7 +55,9 @@ Status BufferOutputStream::Create(int64_t initial_capacity, MemoryPool* pool, BufferOutputStream::~BufferOutputStream() { // This can fail, better to explicitly call close - if (buffer_) { DCHECK(Close().ok()); } + if (buffer_) { + DCHECK(Close().ok()); + } } Status BufferOutputStream::Close() { @@ -102,9 +104,7 @@ Status BufferOutputStream::Reserve(int64_t nbytes) { // ---------------------------------------------------------------------- // OutputStream that doesn't write anything -Status MockOutputStream::Close() { - return Status::OK(); -} +Status MockOutputStream::Close() { return Status::OK(); } Status MockOutputStream::Tell(int64_t* position) { *position = extent_bytes_written_; @@ -158,7 +158,7 @@ Status FixedSizeBufferWriter::Tell(int64_t* position) { Status FixedSizeBufferWriter::Write(const uint8_t* data, int64_t nbytes) { if (nbytes > memcopy_threshold_ && memcopy_num_threads_ > 1) { parallel_memcopy(mutable_data_ + position_, data, nbytes, memcopy_blocksize_, - memcopy_num_threads_); + memcopy_num_threads_); } else { memcpy(mutable_data_ + position_, data, nbytes); } @@ -166,8 +166,8 @@ Status FixedSizeBufferWriter::Write(const uint8_t* data, int64_t nbytes) { return Status::OK(); } -Status FixedSizeBufferWriter::WriteAt( - int64_t position, const uint8_t* data, int64_t nbytes) { +Status FixedSizeBufferWriter::WriteAt(int64_t position, const uint8_t* data, + int64_t nbytes) { std::lock_guard guard(lock_); RETURN_NOT_OK(Seek(position)); return Write(data, nbytes); @@ -206,9 +206,7 @@ Status BufferReader::Tell(int64_t* position) { return Status::OK(); } -bool BufferReader::supports_zero_copy() const { - return true; -} +bool BufferReader::supports_zero_copy() const { return true; } Status BufferReader::Read(int64_t nbytes, int64_t* bytes_read, uint8_t* buffer) { memcpy(buffer, data_ + position_, nbytes); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/memory.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/memory.h b/cpp/src/arrow/io/memory.h index 06384f0..1f81774 100644 --- a/cpp/src/arrow/io/memory.h +++ b/cpp/src/arrow/io/memory.h @@ -45,7 +45,7 @@ class ARROW_EXPORT BufferOutputStream : public OutputStream { explicit BufferOutputStream(const std::shared_ptr& buffer); static Status Create(int64_t initial_capacity, MemoryPool* pool, - std::shared_ptr* out); + std::shared_ptr* out); ~BufferOutputStream(); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/io/test-common.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/io/test-common.h b/cpp/src/arrow/io/test-common.h index 438f378..a4974b7 100644 --- a/cpp/src/arrow/io/test-common.h +++ b/cpp/src/arrow/io/test-common.h @@ -73,8 +73,8 @@ class MemoryMapFixture { tmp_files_.push_back(path); } - Status InitMemoryMap( - int64_t size, const std::string& path, std::shared_ptr* mmap) { + Status InitMemoryMap(int64_t size, const std::string& path, + std::shared_ptr* mmap) { RETURN_NOT_OK(MemoryMappedFile::Create(path, size, mmap)); tmp_files_.push_back(path); return Status::OK(); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/feather-internal.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/feather-internal.h b/cpp/src/arrow/ipc/feather-internal.h index 646c3b2..36cfecc 100644 --- a/cpp/src/arrow/ipc/feather-internal.h +++ b/cpp/src/arrow/ipc/feather-internal.h @@ -49,7 +49,7 @@ struct ARROW_EXPORT ArrayMetadata { ArrayMetadata() {} ArrayMetadata(fbs::Type type, int64_t offset, int64_t length, int64_t null_count, - int64_t total_bytes) + int64_t total_bytes) : type(type), offset(offset), length(length), @@ -135,7 +135,9 @@ class ARROW_EXPORT TableMetadata { bool HasDescription() const { return table_->description() != 0; } std::string GetDescription() const { - if (!HasDescription()) { return std::string(""); } + if (!HasDescription()) { + return std::string(""); + } return table_->description()->str(); } @@ -153,7 +155,7 @@ class ARROW_EXPORT TableMetadata { static inline flatbuffers::Offset GetPrimitiveArray( FBB& fbb, const ArrayMetadata& array) { return fbs::CreatePrimitiveArray(fbb, array.type, fbs::Encoding_PLAIN, array.offset, - array.length, array.null_count, array.total_bytes); + array.length, array.null_count, array.total_bytes); } static inline fbs::TimeUnit ToFlatbufferEnum(TimeUnit::type unit) { http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/feather-test.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/feather-test.cc b/cpp/src/arrow/ipc/feather-test.cc index 029aae3..b76b518 100644 --- a/cpp/src/arrow/ipc/feather-test.cc +++ b/cpp/src/arrow/ipc/feather-test.cc @@ -365,8 +365,8 @@ TEST_F(TestTableWriter, TimeTypes) { ArrayFromVector(is_valid, date_values_vec, &date_array); const auto& prim_values = static_cast(*values); - std::vector> buffers = { - prim_values.null_bitmap(), prim_values.values()}; + std::vector> buffers = {prim_values.null_bitmap(), + prim_values.values()}; std::vector> arrays; arrays.push_back(date_array->data()); @@ -400,7 +400,8 @@ TEST_F(TestTableWriter, PrimitiveNullRoundTrip) { ASSERT_OK(reader_->GetColumn(i, &col)); ASSERT_EQ(batch->column_name(i), col->name()); StringArray str_values(batch->column(i)->length(), nullptr, nullptr, - batch->column(i)->null_bitmap(), batch->column(i)->null_count()); + batch->column(i)->null_bitmap(), + batch->column(i)->null_count()); CheckArrays(str_values, *col->data()->chunk(0)); } } http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/feather.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/feather.cc b/cpp/src/arrow/ipc/feather.cc index 61b96e0..54771d3 100644 --- a/cpp/src/arrow/ipc/feather.cc +++ b/cpp/src/arrow/ipc/feather.cc @@ -61,26 +61,30 @@ static int64_t GetOutputLength(int64_t nbytes) { } static Status WritePadded(io::OutputStream* stream, const uint8_t* data, int64_t length, - int64_t* bytes_written) { + int64_t* bytes_written) { RETURN_NOT_OK(stream->Write(data, length)); int64_t remainder = PaddedLength(length) - length; - if (remainder != 0) { RETURN_NOT_OK(stream->Write(kPaddingBytes, remainder)); } + if (remainder != 0) { + RETURN_NOT_OK(stream->Write(kPaddingBytes, remainder)); + } *bytes_written = length + remainder; return Status::OK(); } /// For compability, we need to write any data sometimes just to keep producing /// files that can be read with an older reader. -static Status WritePaddedBlank( - io::OutputStream* stream, int64_t length, int64_t* bytes_written) { +static Status WritePaddedBlank(io::OutputStream* stream, int64_t length, + int64_t* bytes_written) { const uint8_t null = 0; for (int64_t i = 0; i < length; i++) { RETURN_NOT_OK(stream->Write(&null, 1)); } int64_t remainder = PaddedLength(length) - length; - if (remainder != 0) { RETURN_NOT_OK(stream->Write(kPaddingBytes, remainder)); } + if (remainder != 0) { + RETURN_NOT_OK(stream->Write(kPaddingBytes, remainder)); + } *bytes_written = length + remainder; return Status::OK(); } @@ -90,20 +94,22 @@ static Status WritePaddedBlank( TableBuilder::TableBuilder(int64_t num_rows) : finished_(false), num_rows_(num_rows) {} -FBB& TableBuilder::fbb() { - return fbb_; -} +FBB& TableBuilder::fbb() { return fbb_; } Status TableBuilder::Finish() { - if (finished_) { return Status::Invalid("can only call this once"); } + if (finished_) { + return Status::Invalid("can only call this once"); + } FBString desc = 0; - if (!description_.empty()) { desc = fbb_.CreateString(description_); } + if (!description_.empty()) { + desc = fbb_.CreateString(description_); + } flatbuffers::Offset metadata = 0; - auto root = fbs::CreateCTable( - fbb_, desc, num_rows_, fbb_.CreateVector(columns_), kFeatherVersion, metadata); + auto root = fbs::CreateCTable(fbb_, desc, num_rows_, fbb_.CreateVector(columns_), + kFeatherVersion, metadata); fbb_.Finish(root); finished_ = true; @@ -111,17 +117,15 @@ Status TableBuilder::Finish() { } std::shared_ptr TableBuilder::GetBuffer() const { - return std::make_shared( - fbb_.GetBufferPointer(), static_cast(fbb_.GetSize())); + return std::make_shared(fbb_.GetBufferPointer(), + static_cast(fbb_.GetSize())); } void TableBuilder::SetDescription(const std::string& description) { description_ = description; } -void TableBuilder::SetNumRows(int64_t num_rows) { - num_rows_ = num_rows; -} +void TableBuilder::SetNumRows(int64_t num_rows) { num_rows_ = num_rows; } void TableBuilder::add_column(const flatbuffers::Offset& col) { columns_.push_back(col); @@ -177,21 +181,17 @@ Status ColumnBuilder::Finish() { flatbuffers::Offset metadata = CreateColumnMetadata(); auto column = fbs::CreateColumn(buf, buf.CreateString(name_), values, - ToFlatbufferEnum(type_), // metadata_type - metadata, buf.CreateString(user_metadata_)); + ToFlatbufferEnum(type_), // metadata_type + metadata, buf.CreateString(user_metadata_)); // bad coupling, but OK for now parent_->add_column(column); return Status::OK(); } -void ColumnBuilder::SetValues(const ArrayMetadata& values) { - values_ = values; -} +void ColumnBuilder::SetValues(const ArrayMetadata& values) { values_ = values; } -void ColumnBuilder::SetUserMetadata(const std::string& data) { - user_metadata_ = data; -} +void ColumnBuilder::SetUserMetadata(const std::string& data) { user_metadata_ = data; } void ColumnBuilder::SetCategory(const ArrayMetadata& levels, bool ordered) { type_ = ColumnType::CATEGORY; @@ -209,18 +209,14 @@ void ColumnBuilder::SetTimestamp(TimeUnit::type unit, const std::string& timezon meta_timestamp_.timezone = timezone; } -void ColumnBuilder::SetDate() { - type_ = ColumnType::DATE; -} +void ColumnBuilder::SetDate() { type_ = ColumnType::DATE; } void ColumnBuilder::SetTime(TimeUnit::type unit) { type_ = ColumnType::TIME; meta_time_.unit = unit; } -FBB& ColumnBuilder::fbb() { - return *fbb_; -} +FBB& ColumnBuilder::fbb() { return *fbb_; } std::unique_ptr TableBuilder::AddColumn(const std::string& name) { return std::unique_ptr(new ColumnBuilder(this, name)); @@ -272,7 +268,7 @@ class TableReader::TableReaderImpl { } Status GetDataType(const fbs::PrimitiveArray* values, fbs::TypeMetadata metadata_type, - const void* metadata, std::shared_ptr* out) { + const void* metadata, std::shared_ptr* out) { #define PRIMITIVE_CASE(CAP_TYPE, FACTORY_FUNC) \ case fbs::Type_##CAP_TYPE: \ *out = FACTORY_FUNC(); \ @@ -342,7 +338,7 @@ class TableReader::TableReaderImpl { // @returns: a Buffer instance, the precise type will depend on the kind of // input data source (which may or may not have memory-map like semantics) Status LoadValues(const fbs::PrimitiveArray* meta, fbs::TypeMetadata metadata_type, - const void* metadata, std::shared_ptr* out) { + const void* metadata, std::shared_ptr* out) { std::shared_ptr type; RETURN_NOT_OK(GetDataType(meta, metadata_type, metadata, &type)); @@ -394,8 +390,8 @@ class TableReader::TableReaderImpl { // if (user_meta->size() > 0) { user_metadata_ = user_meta->str(); } std::shared_ptr values; - RETURN_NOT_OK(LoadValues( - col_meta->values(), col_meta->metadata_type(), col_meta->metadata(), &values)); + RETURN_NOT_OK(LoadValues(col_meta->values(), col_meta->metadata_type(), + col_meta->metadata(), &values)); out->reset(new Column(col_meta->name()->str(), values)); return Status::OK(); } @@ -410,41 +406,27 @@ class TableReader::TableReaderImpl { // ---------------------------------------------------------------------- // TableReader public API -TableReader::TableReader() { - impl_.reset(new TableReaderImpl()); -} +TableReader::TableReader() { impl_.reset(new TableReaderImpl()); } TableReader::~TableReader() {} Status TableReader::Open(const std::shared_ptr& source, - std::unique_ptr* out) { + std::unique_ptr* out) { out->reset(new TableReader()); return (*out)->impl_->Open(source); } -bool TableReader::HasDescription() const { - return impl_->HasDescription(); -} +bool TableReader::HasDescription() const { return impl_->HasDescription(); } -std::string TableReader::GetDescription() const { - return impl_->GetDescription(); -} +std::string TableReader::GetDescription() const { return impl_->GetDescription(); } -int TableReader::version() const { - return impl_->version(); -} +int TableReader::version() const { return impl_->version(); } -int64_t TableReader::num_rows() const { - return impl_->num_rows(); -} +int64_t TableReader::num_rows() const { return impl_->num_rows(); } -int64_t TableReader::num_columns() const { - return impl_->num_columns(); -} +int64_t TableReader::num_columns() const { return impl_->num_columns(); } -std::string TableReader::GetColumnName(int i) const { - return impl_->GetColumnName(i); -} +std::string TableReader::GetColumnName(int i) const { return impl_->GetColumnName(i); } Status TableReader::GetColumn(int i, std::shared_ptr* out) { return impl_->GetColumn(i, out); @@ -501,8 +483,8 @@ static Status SanitizeUnsupportedTypes(const Array& values, std::shared_ptr( - values.length(), nullptr, nullptr, values.null_bitmap(), values.null_count()); + *out = std::make_shared(values.length(), nullptr, nullptr, + values.null_bitmap(), values.null_count()); return Status::OK(); } else { return MakeArray(values.data(), out); @@ -537,8 +519,8 @@ class TableWriter::TableWriterImpl : public ArrayVisitor { // Footer: metadata length, magic bytes RETURN_NOT_OK( stream_->Write(reinterpret_cast(&buffer_size), sizeof(uint32_t))); - return stream_->Write( - reinterpret_cast(kFeatherMagicBytes), strlen(kFeatherMagicBytes)); + return stream_->Write(reinterpret_cast(kFeatherMagicBytes), + strlen(kFeatherMagicBytes)); } Status LoadArrayMetadata(const Array& values, ArrayMetadata* meta) { @@ -571,7 +553,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor { // byte boundary, and we write this much data into the stream if (values.null_bitmap()) { RETURN_NOT_OK(WritePadded(stream_.get(), values.null_bitmap()->data(), - values.null_bitmap()->size(), &bytes_written)); + values.null_bitmap()->size(), &bytes_written)); } else { RETURN_NOT_OK(WritePaddedBlank( stream_.get(), BitUtil::BytesForBits(values.length()), &bytes_written)); @@ -592,15 +574,17 @@ class TableWriter::TableWriterImpl : public ArrayVisitor { values_bytes = bin_values.raw_value_offsets()[values.length()]; // Write the variable-length offsets - RETURN_NOT_OK(WritePadded(stream_.get(), - reinterpret_cast(bin_values.raw_value_offsets()), - offset_bytes, &bytes_written)); + RETURN_NOT_OK(WritePadded(stream_.get(), reinterpret_cast( + bin_values.raw_value_offsets()), + offset_bytes, &bytes_written)); } else { RETURN_NOT_OK(WritePaddedBlank(stream_.get(), offset_bytes, &bytes_written)); } meta->total_bytes += bytes_written; - if (bin_values.value_data()) { values_buffer = bin_values.value_data()->data(); } + if (bin_values.value_data()) { + values_buffer = bin_values.value_data()->data(); + } } else { const auto& prim_values = static_cast(values); const auto& fw_type = static_cast(*values.type()); @@ -612,7 +596,9 @@ class TableWriter::TableWriterImpl : public ArrayVisitor { values_bytes = values.length() * fw_type.bit_width() / 8; } - if (prim_values.values()) { values_buffer = prim_values.values()->data(); } + if (prim_values.values()) { + values_buffer = prim_values.values()->data(); + } } if (values_buffer) { RETURN_NOT_OK( @@ -710,9 +696,9 @@ class TableWriter::TableWriterImpl : public ArrayVisitor { Status CheckStarted() { if (!initialized_stream_) { int64_t bytes_written_unused; - RETURN_NOT_OK( - WritePadded(stream_.get(), reinterpret_cast(kFeatherMagicBytes), - strlen(kFeatherMagicBytes), &bytes_written_unused)); + RETURN_NOT_OK(WritePadded(stream_.get(), + reinterpret_cast(kFeatherMagicBytes), + strlen(kFeatherMagicBytes), &bytes_written_unused)); initialized_stream_ = true; } return Status::OK(); @@ -728,33 +714,25 @@ class TableWriter::TableWriterImpl : public ArrayVisitor { Status AppendPrimitive(const PrimitiveArray& values, ArrayMetadata* out); }; -TableWriter::TableWriter() { - impl_.reset(new TableWriterImpl()); -} +TableWriter::TableWriter() { impl_.reset(new TableWriterImpl()); } TableWriter::~TableWriter() {} -Status TableWriter::Open( - const std::shared_ptr& stream, std::unique_ptr* out) { +Status TableWriter::Open(const std::shared_ptr& stream, + std::unique_ptr* out) { out->reset(new TableWriter()); return (*out)->impl_->Open(stream); } -void TableWriter::SetDescription(const std::string& desc) { - impl_->SetDescription(desc); -} +void TableWriter::SetDescription(const std::string& desc) { impl_->SetDescription(desc); } -void TableWriter::SetNumRows(int64_t num_rows) { - impl_->SetNumRows(num_rows); -} +void TableWriter::SetNumRows(int64_t num_rows) { impl_->SetNumRows(num_rows); } Status TableWriter::Append(const std::string& name, const Array& values) { return impl_->Append(name, values); } -Status TableWriter::Finalize() { - return impl_->Finalize(); -} +Status TableWriter::Finalize() { return impl_->Finalize(); } } // namespace feather } // namespace ipc http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/feather.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/feather.h b/cpp/src/arrow/ipc/feather.h index 4d59a8b..8abcb5c 100644 --- a/cpp/src/arrow/ipc/feather.h +++ b/cpp/src/arrow/ipc/feather.h @@ -56,7 +56,7 @@ class ARROW_EXPORT TableReader { ~TableReader(); static Status Open(const std::shared_ptr& source, - std::unique_ptr* out); + std::unique_ptr* out); // Optional table description // @@ -83,8 +83,8 @@ class ARROW_EXPORT TableWriter { public: ~TableWriter(); - static Status Open( - const std::shared_ptr& stream, std::unique_ptr* out); + static Status Open(const std::shared_ptr& stream, + std::unique_ptr* out); void SetDescription(const std::string& desc); void SetNumRows(int64_t num_rows); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/file-to-stream.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/file-to-stream.cc b/cpp/src/arrow/ipc/file-to-stream.cc index a1feedc..4707c4f 100644 --- a/cpp/src/arrow/ipc/file-to-stream.cc +++ b/cpp/src/arrow/ipc/file-to-stream.cc @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +#include #include "arrow/io/file.h" #include "arrow/ipc/reader.h" #include "arrow/ipc/writer.h" #include "arrow/status.h" -#include #include "arrow/util/io-util.h" http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/ipc-json-test.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/ipc-json-test.cc b/cpp/src/arrow/ipc/ipc-json-test.cc index 79344df..35264fa 100644 --- a/cpp/src/arrow/ipc/ipc-json-test.cc +++ b/cpp/src/arrow/ipc/ipc-json-test.cc @@ -77,7 +77,9 @@ void TestArrayRoundTrip(const Array& array) { rj::Document d; d.Parse(array_as_json); - if (d.HasParseError()) { FAIL() << "JSON parsing failed"; } + if (d.HasParseError()) { + FAIL() << "JSON parsing failed"; + } std::shared_ptr out; ASSERT_OK(internal::ReadArray(default_memory_pool(), d, array.type(), &out)); @@ -88,7 +90,8 @@ void TestArrayRoundTrip(const Array& array) { template void CheckPrimitive(const std::shared_ptr& type, - const std::vector& is_valid, const std::vector& values) { + const std::vector& is_valid, + const std::vector& values) { MemoryPool* pool = default_memory_pool(); typename TypeTraits::BuilderType builder(pool); @@ -108,16 +111,17 @@ void CheckPrimitive(const std::shared_ptr& type, TEST(TestJsonSchemaWriter, FlatTypes) { // TODO // field("f14", date32()) - std::vector> fields = {field("f0", int8()), - field("f1", int16(), false), field("f2", int32()), field("f3", int64(), false), - field("f4", uint8()), field("f5", uint16()), field("f6", uint32()), - field("f7", uint64()), field("f8", float32()), field("f9", float64()), - field("f10", utf8()), field("f11", binary()), field("f12", list(int32())), + std::vector> fields = { + field("f0", int8()), field("f1", int16(), false), field("f2", int32()), + field("f3", int64(), false), field("f4", uint8()), field("f5", uint16()), + field("f6", uint32()), field("f7", uint64()), field("f8", float32()), + field("f9", float64()), field("f10", utf8()), field("f11", binary()), + field("f12", list(int32())), field("f13", struct_({field("s1", int32()), field("s2", utf8())})), field("f15", date64()), field("f16", timestamp(TimeUnit::NANO)), field("f17", time64(TimeUnit::MICRO)), field("f18", union_({field("u1", int8()), field("u2", time32(TimeUnit::MILLI))}, - {0, 1}, UnionMode::DENSE))}; + {0, 1}, UnionMode::DENSE))}; Schema schema(fields); TestSchemaRoundTrip(schema); @@ -185,8 +189,8 @@ TEST(TestJsonArrayWriter, NestedTypes) { struct_({field("f1", int32()), field("f2", int32()), field("f3", int32())}); std::vector> fields = {values_array, values_array, values_array}; - StructArray struct_array( - struct_type, static_cast(struct_is_valid.size()), fields, struct_bitmap, 2); + StructArray struct_array(struct_type, static_cast(struct_is_valid.size()), fields, + struct_bitmap, 2); TestArrayRoundTrip(struct_array); } @@ -202,7 +206,7 @@ TEST(TestJsonArrayWriter, Unions) { // Data generation for test case below void MakeBatchArrays(const std::shared_ptr& schema, const int num_rows, - std::vector>* arrays) { + std::vector>* arrays) { std::vector is_valid; test::random_is_valid(num_rows, 0.25, &is_valid); @@ -266,8 +270,8 @@ TEST(TestJsonFileReadWrite, BasicRoundTrip) { std::unique_ptr reader; - auto buffer = std::make_shared( - reinterpret_cast(result.c_str()), static_cast(result.size())); + auto buffer = std::make_shared(reinterpret_cast(result.c_str()), + static_cast(result.size())); ASSERT_OK(JsonReader::Open(buffer, &reader)); ASSERT_TRUE(reader->schema()->Equals(*schema)); @@ -332,8 +336,8 @@ TEST(TestJsonFileReadWrite, MinimalFormatExample) { } )example"; - auto buffer = std::make_shared( - reinterpret_cast(example), strlen(example)); + auto buffer = std::make_shared(reinterpret_cast(example), + strlen(example)); std::unique_ptr reader; ASSERT_OK(JsonReader::Open(buffer, &reader)); @@ -361,9 +365,9 @@ TEST(TestJsonFileReadWrite, MinimalFormatExample) { #define BATCH_CASES() \ ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \ - &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch, \ - &MakeStruct, &MakeUnion, &MakeDates, &MakeTimestamps, &MakeTimes, &MakeFWBinary, \ - &MakeDictionary); + &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, \ + &MakeStringTypesRecordBatch, &MakeStruct, &MakeUnion, &MakeDates, \ + &MakeTimestamps, &MakeTimes, &MakeFWBinary, &MakeDictionary); class TestJsonRoundTrip : public ::testing::TestWithParam { public: @@ -382,7 +386,7 @@ void CheckRoundtrip(const RecordBatch& batch) { ASSERT_OK(writer->Finish(&result)); auto buffer = std::make_shared(reinterpret_cast(result.c_str()), - static_cast(result.size())); + static_cast(result.size())); std::unique_ptr reader; ASSERT_OK(JsonReader::Open(buffer, &reader)); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc b/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc index c890d82..a88120a 100644 --- a/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc @@ -80,7 +80,7 @@ static void BM_WriteRecordBatch(benchmark::State& state) { // NOLINT non-const int32_t metadata_length; int64_t body_length; if (!ipc::WriteRecordBatch(*record_batch, 0, &stream, &metadata_length, &body_length, - default_memory_pool()) + default_memory_pool()) .ok()) { state.SkipWithError("Failed to write!"); } @@ -101,7 +101,7 @@ static void BM_ReadRecordBatch(benchmark::State& state) { // NOLINT non-const r int32_t metadata_length; int64_t body_length; if (!ipc::WriteRecordBatch(*record_batch, 0, &stream, &metadata_length, &body_length, - default_memory_pool()) + default_memory_pool()) .ok()) { state.SkipWithError("Failed to write!"); } http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/ipc-read-write-test.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc index 2119ff7..6c70517 100644 --- a/cpp/src/arrow/ipc/ipc-read-write-test.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc @@ -126,40 +126,45 @@ TEST_F(TestSchemaMetadata, NestedFields) { CheckRoundtrip(schema, &memo); } -#define BATCH_CASES() \ - ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \ - &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch, \ - &MakeStruct, &MakeUnion, &MakeDictionary, &MakeDates, &MakeTimestamps, &MakeTimes, \ - &MakeFWBinary, &MakeBooleanBatch); +#define BATCH_CASES() \ + ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \ + &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, \ + &MakeStringTypesRecordBatch, &MakeStruct, &MakeUnion, \ + &MakeDictionary, &MakeDates, &MakeTimestamps, &MakeTimes, \ + &MakeFWBinary, &MakeBooleanBatch); static int g_file_number = 0; class IpcTestFixture : public io::MemoryMapFixture { public: Status DoStandardRoundTrip(const RecordBatch& batch, bool zero_data, - std::shared_ptr* batch_result) { + std::shared_ptr* batch_result) { int32_t metadata_length; int64_t body_length; const int64_t buffer_offset = 0; - if (zero_data) { RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); } + if (zero_data) { + RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); + } RETURN_NOT_OK(mmap_->Seek(0)); - RETURN_NOT_OK(WriteRecordBatch( - batch, buffer_offset, mmap_.get(), &metadata_length, &body_length, pool_)); + RETURN_NOT_OK(WriteRecordBatch(batch, buffer_offset, mmap_.get(), &metadata_length, + &body_length, pool_)); std::unique_ptr message; RETURN_NOT_OK(ReadMessage(0, metadata_length, mmap_.get(), &message)); io::BufferReader buffer_reader(message->body()); - return ReadRecordBatch( - *message->metadata(), batch.schema(), &buffer_reader, batch_result); + return ReadRecordBatch(*message->metadata(), batch.schema(), &buffer_reader, + batch_result); } - Status DoLargeRoundTrip( - const RecordBatch& batch, bool zero_data, std::shared_ptr* result) { - if (zero_data) { RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); } + Status DoLargeRoundTrip(const RecordBatch& batch, bool zero_data, + std::shared_ptr* result) { + if (zero_data) { + RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); + } RETURN_NOT_OK(mmap_->Seek(0)); std::shared_ptr file_writer; @@ -244,8 +249,8 @@ TEST_F(TestIpcRoundTrip, MetadataVersion) { const int64_t buffer_offset = 0; - ASSERT_OK(WriteRecordBatch( - *batch, buffer_offset, mmap_.get(), &metadata_length, &body_length, pool_)); + ASSERT_OK(WriteRecordBatch(*batch, buffer_offset, mmap_.get(), &metadata_length, + &body_length, pool_)); std::unique_ptr message; ASSERT_OK(ReadMessage(0, metadata_length, mmap_.get(), &message)); @@ -258,7 +263,9 @@ TEST_P(TestIpcRoundTrip, SliceRoundTrip) { ASSERT_OK((*GetParam())(&batch)); // NOLINT clang-tidy gtest issue // Skip the zero-length case - if (batch->num_rows() < 2) { return; } + if (batch->num_rows() < 2) { + return; + } auto sliced_batch = batch->Slice(2, 10); CheckRoundtrip(*sliced_batch, 1 << 20); @@ -282,8 +289,9 @@ TEST_P(TestIpcRoundTrip, ZeroLengthArrays) { ASSERT_OK(AllocateBuffer(pool_, sizeof(int32_t), &value_offsets)); *reinterpret_cast(value_offsets->mutable_data()) = 0; - std::shared_ptr bin_array = std::make_shared(0, value_offsets, - std::make_shared(nullptr, 0), std::make_shared(nullptr, 0)); + std::shared_ptr bin_array = std::make_shared( + 0, value_offsets, std::make_shared(nullptr, 0), + std::make_shared(nullptr, 0)); // null value_offsets std::shared_ptr bin_array2 = std::make_shared(0, nullptr, nullptr); @@ -357,8 +365,8 @@ TEST_F(TestWriteRecordBatch, SliceTruncatesBuffers) { std::shared_ptr offsets_buffer; ASSERT_OK( test::CopyBufferFromVector(type_offsets, default_memory_pool(), &offsets_buffer)); - a1 = std::make_shared( - dense_union_type, a0->length(), struct_children, ids_buffer, offsets_buffer); + a1 = std::make_shared(dense_union_type, a0->length(), struct_children, + ids_buffer, offsets_buffer); CheckArray(a1); } @@ -367,8 +375,8 @@ void TestGetRecordBatchSize(std::shared_ptr batch) { int32_t mock_metadata_length = -1; int64_t mock_body_length = -1; int64_t size = -1; - ASSERT_OK(WriteRecordBatch( - *batch, 0, &mock, &mock_metadata_length, &mock_body_length, default_memory_pool())); + ASSERT_OK(WriteRecordBatch(*batch, 0, &mock, &mock_metadata_length, &mock_body_length, + default_memory_pool())); ASSERT_OK(GetRecordBatchSize(*batch, &size)); ASSERT_EQ(mock.GetExtentBytesWritten(), size); } @@ -398,8 +406,8 @@ class RecursionLimits : public ::testing::Test, public io::MemoryMapFixture { void TearDown() { io::MemoryMapFixture::TearDown(); } Status WriteToMmap(int recursion_level, bool override_level, int32_t* metadata_length, - int64_t* body_length, std::shared_ptr* batch, - std::shared_ptr* schema) { + int64_t* body_length, std::shared_ptr* batch, + std::shared_ptr* schema) { const int batch_length = 5; TypePtr type = int32(); std::shared_ptr array; @@ -425,10 +433,10 @@ class RecursionLimits : public ::testing::Test, public io::MemoryMapFixture { if (override_level) { return WriteRecordBatch(**batch, 0, mmap_.get(), metadata_length, body_length, - pool_, recursion_level + 1); + pool_, recursion_level + 1); } else { - return WriteRecordBatch( - **batch, 0, mmap_.get(), metadata_length, body_length, pool_); + return WriteRecordBatch(**batch, 0, mmap_.get(), metadata_length, body_length, + pool_); } } @@ -442,8 +450,8 @@ TEST_F(RecursionLimits, WriteLimit) { int64_t body_length = -1; std::shared_ptr schema; std::shared_ptr batch; - ASSERT_RAISES(Invalid, - WriteToMmap((1 << 8) + 1, false, &metadata_length, &body_length, &batch, &schema)); + ASSERT_RAISES(Invalid, WriteToMmap((1 << 8) + 1, false, &metadata_length, &body_length, + &batch, &schema)); } TEST_F(RecursionLimits, ReadLimit) { @@ -454,8 +462,8 @@ TEST_F(RecursionLimits, ReadLimit) { const int recursion_depth = 64; std::shared_ptr batch; - ASSERT_OK(WriteToMmap( - recursion_depth, true, &metadata_length, &body_length, &batch, &schema)); + ASSERT_OK(WriteToMmap(recursion_depth, true, &metadata_length, &body_length, &batch, + &schema)); std::unique_ptr message; ASSERT_OK(ReadMessage(0, metadata_length, mmap_.get(), &message)); @@ -472,16 +480,16 @@ TEST_F(RecursionLimits, StressLimit) { int64_t body_length = -1; std::shared_ptr schema; std::shared_ptr batch; - ASSERT_OK(WriteToMmap( - recursion_depth, true, &metadata_length, &body_length, &batch, &schema)); + ASSERT_OK(WriteToMmap(recursion_depth, true, &metadata_length, &body_length, &batch, + &schema)); std::unique_ptr message; ASSERT_OK(ReadMessage(0, metadata_length, mmap_.get(), &message)); io::BufferReader reader(message->body()); std::shared_ptr result; - ASSERT_OK(ReadRecordBatch( - *message->metadata(), schema, recursion_depth + 1, &reader, &result)); + ASSERT_OK(ReadRecordBatch(*message->metadata(), schema, recursion_depth + 1, &reader, + &result)); *it_works = result->Equals(*batch); }; @@ -568,8 +576,8 @@ class TestStreamFormat : public ::testing::TestWithParam { } void TearDown() {} - Status RoundTripHelper( - const RecordBatch& batch, std::vector>* out_batches) { + Status RoundTripHelper(const RecordBatch& batch, + std::vector>* out_batches) { // Write the file std::shared_ptr writer; RETURN_NOT_OK(RecordBatchStreamWriter::Open(sink_.get(), batch.schema(), &writer)); @@ -589,7 +597,9 @@ class TestStreamFormat : public ::testing::TestWithParam { std::shared_ptr chunk; while (true) { RETURN_NOT_OK(reader->ReadNextRecordBatch(&chunk)); - if (chunk == nullptr) { break; } + if (chunk == nullptr) { + break; + } out_batches->emplace_back(chunk); } return Status::OK(); @@ -747,8 +757,8 @@ TEST_F(TestTensorRoundTrip, NonContiguous) { int32_t metadata_length; int64_t body_length; ASSERT_OK(mmap_->Seek(0)); - ASSERT_RAISES( - Invalid, WriteTensor(tensor, mmap_.get(), &metadata_length, &body_length)); + ASSERT_RAISES(Invalid, + WriteTensor(tensor, mmap_.get(), &metadata_length, &body_length)); } } // namespace ipc http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/json-integration-test.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/json-integration-test.cc b/cpp/src/arrow/ipc/json-integration-test.cc index 18f5dfa..035f708 100644 --- a/cpp/src/arrow/ipc/json-integration-test.cc +++ b/cpp/src/arrow/ipc/json-integration-test.cc @@ -40,7 +40,8 @@ DEFINE_string(arrow, "", "Arrow file name"); DEFINE_string(json, "", "JSON file name"); -DEFINE_string(mode, "VALIDATE", +DEFINE_string( + mode, "VALIDATE", "Mode of integration testing tool (ARROW_TO_JSON, JSON_TO_ARROW, VALIDATE)"); DEFINE_bool(integration, false, "Run in integration test mode"); DEFINE_bool(verbose, true, "Verbose output"); @@ -55,8 +56,8 @@ bool file_exists(const char* path) { } // Convert JSON file to IPC binary format -static Status ConvertJsonToArrow( - const std::string& json_path, const std::string& arrow_path) { +static Status ConvertJsonToArrow(const std::string& json_path, + const std::string& arrow_path) { std::shared_ptr in_file; std::shared_ptr out_file; @@ -89,8 +90,8 @@ static Status ConvertJsonToArrow( } // Convert IPC binary format to JSON -static Status ConvertArrowToJson( - const std::string& arrow_path, const std::string& json_path) { +static Status ConvertArrowToJson(const std::string& arrow_path, + const std::string& json_path) { std::shared_ptr in_file; std::shared_ptr out_file; @@ -116,11 +117,11 @@ static Status ConvertArrowToJson( std::string result; RETURN_NOT_OK(writer->Finish(&result)); return out_file->Write(reinterpret_cast(result.c_str()), - static_cast(result.size())); + static_cast(result.size())); } -static Status ValidateArrowVsJson( - const std::string& arrow_path, const std::string& json_path) { +static Status ValidateArrowVsJson(const std::string& arrow_path, + const std::string& json_path) { // Construct JSON reader std::shared_ptr json_file; RETURN_NOT_OK(io::ReadableFile::Open(json_path, &json_file)); @@ -151,7 +152,9 @@ static Status ValidateArrowVsJson( << "Arrow schema: \n" << arrow_schema->ToString(); - if (FLAGS_verbose) { std::cout << ss.str() << std::endl; } + if (FLAGS_verbose) { + std::cout << ss.str() << std::endl; + } return Status::Invalid("Schemas did not match"); } @@ -188,10 +191,14 @@ static Status ValidateArrowVsJson( } Status RunCommand(const std::string& json_path, const std::string& arrow_path, - const std::string& command) { - if (json_path == "") { return Status::Invalid("Must specify json file name"); } + const std::string& command) { + if (json_path == "") { + return Status::Invalid("Must specify json file name"); + } - if (arrow_path == "") { return Status::Invalid("Must specify arrow file name"); } + if (arrow_path == "") { + return Status::Invalid("Must specify arrow file name"); + } if (command == "ARROW_TO_JSON") { if (!file_exists(arrow_path.c_str())) { @@ -240,8 +247,8 @@ class TestJSONIntegration : public ::testing::Test { do { std::shared_ptr out; RETURN_NOT_OK(io::FileOutputStream::Open(path, &out)); - RETURN_NOT_OK(out->Write( - reinterpret_cast(data), static_cast(strlen(data)))); + RETURN_NOT_OK(out->Write(reinterpret_cast(data), + static_cast(strlen(data)))); } while (0); return Status::OK(); } http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/json-internal.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/json-internal.cc b/cpp/src/arrow/ipc/json-internal.cc index 69e4ae8..175d75b 100644 --- a/cpp/src/arrow/ipc/json-internal.cc +++ b/cpp/src/arrow/ipc/json-internal.cc @@ -199,7 +199,7 @@ class SchemaWriter { typename std::enable_if::value || std::is_base_of::value || std::is_base_of::value, - void>::type + void>::type WriteTypeMetadata(const T& type) {} void WriteTypeMetadata(const Integer& type) { @@ -508,7 +508,7 @@ class ArrayWriter { } Status WriteChildren(const std::vector>& fields, - const std::vector>& arrays) { + const std::vector>& arrays) { writer_->Key("children"); writer_->StartArray(); for (size_t i = 0; i < fields.size(); ++i) { @@ -602,16 +602,16 @@ static Status GetObjectBool(const RjObject& obj, const std::string& key, bool* o return Status::OK(); } -static Status GetObjectString( - const RjObject& obj, const std::string& key, std::string* out) { +static Status GetObjectString(const RjObject& obj, const std::string& key, + std::string* out) { const auto& it = obj.FindMember(key); RETURN_NOT_STRING(key, it, obj); *out = it->value.GetString(); return Status::OK(); } -static Status GetInteger( - const rj::Value::ConstObject& json_type, std::shared_ptr* type) { +static Status GetInteger(const rj::Value::ConstObject& json_type, + std::shared_ptr* type) { const auto& it_bit_width = json_type.FindMember("bitWidth"); RETURN_NOT_INT("bitWidth", it_bit_width, json_type); @@ -642,8 +642,8 @@ static Status GetInteger( return Status::OK(); } -static Status GetFloatingPoint( - const RjObject& json_type, std::shared_ptr* type) { +static Status GetFloatingPoint(const RjObject& json_type, + std::shared_ptr* type) { const auto& it_precision = json_type.FindMember("precision"); RETURN_NOT_STRING("precision", it_precision, json_type); @@ -663,8 +663,8 @@ static Status GetFloatingPoint( return Status::OK(); } -static Status GetFixedSizeBinary( - const RjObject& json_type, std::shared_ptr* type) { +static Status GetFixedSizeBinary(const RjObject& json_type, + std::shared_ptr* type) { const auto& it_byte_width = json_type.FindMember("byteWidth"); RETURN_NOT_INT("byteWidth", it_byte_width, json_type); @@ -756,8 +756,8 @@ static Status GetTimestamp(const RjObject& json_type, std::shared_ptr* } static Status GetUnion(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { + const std::vector>& children, + std::shared_ptr* type) { const auto& it_mode = json_type.FindMember("mode"); RETURN_NOT_STRING("mode", it_mode, json_type); @@ -790,8 +790,8 @@ static Status GetUnion(const RjObject& json_type, } static Status GetType(const RjObject& json_type, - const std::vector>& children, - std::shared_ptr* type) { + const std::vector>& children, + std::shared_ptr* type) { const auto& it_type_name = json_type.FindMember("name"); RETURN_NOT_STRING("name", it_type_name, json_type); @@ -831,10 +831,11 @@ static Status GetType(const RjObject& json_type, } static Status GetField(const rj::Value& obj, const DictionaryMemo* dictionary_memo, - std::shared_ptr* field); + std::shared_ptr* field); static Status GetFieldsFromArray(const rj::Value& obj, - const DictionaryMemo* dictionary_memo, std::vector>* fields) { + const DictionaryMemo* dictionary_memo, + std::vector>* fields) { const auto& values = obj.GetArray(); fields->resize(values.Size()); @@ -845,7 +846,7 @@ static Status GetFieldsFromArray(const rj::Value& obj, } static Status ParseDictionary(const RjObject& obj, int64_t* id, bool* is_ordered, - std::shared_ptr* index_type) { + std::shared_ptr* index_type) { int32_t int32_id; RETURN_NOT_OK(GetObjectInt(obj, "id", &int32_id)); *id = int32_id; @@ -866,8 +867,10 @@ static Status ParseDictionary(const RjObject& obj, int64_t* id, bool* is_ordered } static Status GetField(const rj::Value& obj, const DictionaryMemo* dictionary_memo, - std::shared_ptr* field) { - if (!obj.IsObject()) { return Status::Invalid("Field was not a JSON object"); } + std::shared_ptr* field) { + if (!obj.IsObject()) { + return Status::Invalid("Field was not a JSON object"); + } const auto& json_field = obj.GetObject(); std::string name; @@ -884,8 +887,8 @@ static Status GetField(const rj::Value& obj, const DictionaryMemo* dictionary_me int64_t dictionary_id; bool is_ordered; std::shared_ptr index_type; - RETURN_NOT_OK(ParseDictionary( - it_dictionary->value.GetObject(), &dictionary_id, &is_ordered, &index_type)); + RETURN_NOT_OK(ParseDictionary(it_dictionary->value.GetObject(), &dictionary_id, + &is_ordered, &index_type)); std::shared_ptr dictionary; RETURN_NOT_OK(dictionary_memo->GetDictionary(dictionary_id, &dictionary)); @@ -941,13 +944,13 @@ UnboxValue(const rj::Value& val) { class ArrayReader { public: explicit ArrayReader(const rj::Value& json_array, const std::shared_ptr& type, - MemoryPool* pool) + MemoryPool* pool) : json_array_(json_array), type_(type), pool_(pool) {} Status ParseTypeValues(const DataType& type); Status GetValidityBuffer(const std::vector& is_valid, int32_t* null_count, - std::shared_ptr* validity_buffer) { + std::shared_ptr* validity_buffer) { int length = static_cast(is_valid.size()); std::shared_ptr out_buffer; @@ -1024,7 +1027,9 @@ class ArrayReader { DCHECK(hex_string.size() % 2 == 0) << "Expected base16 hex string"; int32_t length = static_cast(hex_string.size()) / 2; - if (byte_buffer->size() < length) { RETURN_NOT_OK(byte_buffer->Resize(length)); } + if (byte_buffer->size() < length) { + RETURN_NOT_OK(byte_buffer->Resize(length)); + } const char* hex_data = hex_string.c_str(); uint8_t* byte_buffer_data = byte_buffer->mutable_data(); @@ -1078,8 +1083,8 @@ class ArrayReader { } template - Status GetIntArray( - const RjArray& json_array, const int32_t length, std::shared_ptr* out) { + Status GetIntArray(const RjArray& json_array, const int32_t length, + std::shared_ptr* out) { std::shared_ptr buffer; RETURN_NOT_OK(AllocateBuffer(pool_, length * sizeof(T), &buffer)); @@ -1102,15 +1107,15 @@ class ArrayReader { const auto& json_offsets = obj_->FindMember("OFFSET"); RETURN_NOT_ARRAY("OFFSET", json_offsets, *obj_); std::shared_ptr offsets_buffer; - RETURN_NOT_OK(GetIntArray( - json_offsets->value.GetArray(), length_ + 1, &offsets_buffer)); + RETURN_NOT_OK(GetIntArray(json_offsets->value.GetArray(), length_ + 1, + &offsets_buffer)); std::vector> children; RETURN_NOT_OK(GetChildren(*obj_, type, &children)); DCHECK_EQ(children.size(), 1); - result_ = std::make_shared( - type_, length_, offsets_buffer, children[0], validity_buffer, null_count); + result_ = std::make_shared(type_, length_, offsets_buffer, children[0], + validity_buffer, null_count); return Status::OK(); } @@ -1123,8 +1128,8 @@ class ArrayReader { std::vector> fields; RETURN_NOT_OK(GetChildren(*obj_, type, &fields)); - result_ = std::make_shared( - type_, length_, fields, validity_buffer, null_count); + result_ = std::make_shared(type_, length_, fields, validity_buffer, + null_count); return Status::OK(); } @@ -1154,7 +1159,7 @@ class ArrayReader { RETURN_NOT_OK(GetChildren(*obj_, type, &children)); result_ = std::make_shared(type_, length_, children, type_id_buffer, - offsets_buffer, validity_buffer, null_count); + offsets_buffer, validity_buffer, null_count); return Status::OK(); } @@ -1177,7 +1182,7 @@ class ArrayReader { } Status GetChildren(const RjObject& obj, const DataType& type, - std::vector>* array) { + std::vector>* array) { const auto& json_children = obj.FindMember("children"); RETURN_NOT_ARRAY("children", json_children, obj); const auto& json_children_arr = json_children->value.GetArray(); @@ -1280,7 +1285,8 @@ static Status GetDictionaryTypes(const RjArray& fields, DictionaryTypeMap* id_to } static Status ReadDictionary(const RjObject& obj, const DictionaryTypeMap& id_to_field, - MemoryPool* pool, int64_t* dictionary_id, std::shared_ptr* out) { + MemoryPool* pool, int64_t* dictionary_id, + std::shared_ptr* out) { int id; RETURN_NOT_OK(GetObjectInt(obj, "id", &id)); @@ -1312,7 +1318,7 @@ static Status ReadDictionary(const RjObject& obj, const DictionaryTypeMap& id_to } static Status ReadDictionaries(const rj::Value& doc, const DictionaryTypeMap& id_to_field, - MemoryPool* pool, DictionaryMemo* dictionary_memo) { + MemoryPool* pool, DictionaryMemo* dictionary_memo) { auto it = doc.FindMember("dictionaries"); if (it == doc.MemberEnd()) { // No dictionaries @@ -1334,8 +1340,8 @@ static Status ReadDictionaries(const rj::Value& doc, const DictionaryTypeMap& id return Status::OK(); } -Status ReadSchema( - const rj::Value& json_schema, MemoryPool* pool, std::shared_ptr* schema) { +Status ReadSchema(const rj::Value& json_schema, MemoryPool* pool, + std::shared_ptr* schema) { auto it = json_schema.FindMember("schema"); RETURN_NOT_OBJECT("schema", it, json_schema); const auto& obj_schema = it->value.GetObject(); @@ -1359,7 +1365,7 @@ Status ReadSchema( } Status ReadRecordBatch(const rj::Value& json_obj, const std::shared_ptr& schema, - MemoryPool* pool, std::shared_ptr* batch) { + MemoryPool* pool, std::shared_ptr* batch) { DCHECK(json_obj.IsObject()); const auto& batch_obj = json_obj.GetObject(); @@ -1409,14 +1415,16 @@ Status WriteArray(const std::string& name, const Array& array, RjWriter* json_wr } Status ReadArray(MemoryPool* pool, const rj::Value& json_array, - const std::shared_ptr& type, std::shared_ptr* array) { + const std::shared_ptr& type, std::shared_ptr* array) { ArrayReader converter(json_array, type, pool); return converter.GetArray(array); } Status ReadArray(MemoryPool* pool, const rj::Value& json_array, const Schema& schema, - std::shared_ptr* array) { - if (!json_array.IsObject()) { return Status::Invalid("Element was not a JSON object"); } + std::shared_ptr* array) { + if (!json_array.IsObject()) { + return Status::Invalid("Element was not a JSON object"); + } const auto& json_obj = json_array.GetObject(); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/json-internal.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/json-internal.h b/cpp/src/arrow/ipc/json-internal.h index 5571d92..9b641cd 100644 --- a/cpp/src/arrow/ipc/json-internal.h +++ b/cpp/src/arrow/ipc/json-internal.h @@ -99,17 +99,17 @@ Status WriteSchema(const Schema& schema, RjWriter* writer); Status WriteRecordBatch(const RecordBatch& batch, RjWriter* writer); Status WriteArray(const std::string& name, const Array& array, RjWriter* writer); -Status ReadSchema( - const rj::Value& json_obj, MemoryPool* pool, std::shared_ptr* schema); +Status ReadSchema(const rj::Value& json_obj, MemoryPool* pool, + std::shared_ptr* schema); Status ReadRecordBatch(const rj::Value& json_obj, const std::shared_ptr& schema, - MemoryPool* pool, std::shared_ptr* batch); + MemoryPool* pool, std::shared_ptr* batch); Status ReadArray(MemoryPool* pool, const rj::Value& json_obj, - const std::shared_ptr& type, std::shared_ptr* array); + const std::shared_ptr& type, std::shared_ptr* array); Status ReadArray(MemoryPool* pool, const rj::Value& json_obj, const Schema& schema, - std::shared_ptr* array); + std::shared_ptr* array); } // namespace internal } // namespace json http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/json.cc ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/json.cc b/cpp/src/arrow/ipc/json.cc index 36e343e..f57101a 100644 --- a/cpp/src/arrow/ipc/json.cc +++ b/cpp/src/arrow/ipc/json.cc @@ -79,15 +79,13 @@ JsonWriter::JsonWriter(const std::shared_ptr& schema) { JsonWriter::~JsonWriter() {} -Status JsonWriter::Open( - const std::shared_ptr& schema, std::unique_ptr* writer) { +Status JsonWriter::Open(const std::shared_ptr& schema, + std::unique_ptr* writer) { *writer = std::unique_ptr(new JsonWriter(schema)); return (*writer)->impl_->Start(); } -Status JsonWriter::Finish(std::string* result) { - return impl_->Finish(result); -} +Status JsonWriter::Finish(std::string* result) { return impl_->Finish(result); } Status JsonWriter::WriteRecordBatch(const RecordBatch& batch) { return impl_->WriteRecordBatch(batch); @@ -103,8 +101,10 @@ class JsonReader::JsonReaderImpl { Status ParseAndReadSchema() { doc_.Parse(reinterpret_cast(data_->data()), - static_cast(data_->size())); - if (doc_.HasParseError()) { return Status::IOError("JSON parsing failed"); } + static_cast(data_->size())); + if (doc_.HasParseError()) { + return Status::IOError("JSON parsing failed"); + } RETURN_NOT_OK(json::internal::ReadSchema(doc_, pool_, &schema_)); @@ -120,8 +120,8 @@ class JsonReader::JsonReaderImpl { DCHECK_LT(i, static_cast(record_batches_->GetArray().Size())) << "i out of bounds"; - return json::internal::ReadRecordBatch( - record_batches_->GetArray()[i], schema_, pool_, batch); + return json::internal::ReadRecordBatch(record_batches_->GetArray()[i], schema_, pool_, + batch); } std::shared_ptr schema() const { return schema_; } @@ -145,24 +145,20 @@ JsonReader::JsonReader(MemoryPool* pool, const std::shared_ptr& data) { JsonReader::~JsonReader() {} -Status JsonReader::Open( - const std::shared_ptr& data, std::unique_ptr* reader) { +Status JsonReader::Open(const std::shared_ptr& data, + std::unique_ptr* reader) { return Open(default_memory_pool(), data, reader); } Status JsonReader::Open(MemoryPool* pool, const std::shared_ptr& data, - std::unique_ptr* reader) { + std::unique_ptr* reader) { *reader = std::unique_ptr(new JsonReader(pool, data)); return (*reader)->impl_->ParseAndReadSchema(); } -std::shared_ptr JsonReader::schema() const { - return impl_->schema(); -} +std::shared_ptr JsonReader::schema() const { return impl_->schema(); } -int JsonReader::num_record_batches() const { - return impl_->num_record_batches(); -} +int JsonReader::num_record_batches() const { return impl_->num_record_batches(); } Status JsonReader::ReadRecordBatch(int i, std::shared_ptr* batch) const { return impl_->ReadRecordBatch(i, batch); http://git-wip-us.apache.org/repos/asf/arrow/blob/07b89bf3/cpp/src/arrow/ipc/json.h ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/json.h b/cpp/src/arrow/ipc/json.h index 2ba27c7..be26f02 100644 --- a/cpp/src/arrow/ipc/json.h +++ b/cpp/src/arrow/ipc/json.h @@ -41,8 +41,8 @@ class ARROW_EXPORT JsonWriter { public: ~JsonWriter(); - static Status Open( - const std::shared_ptr& schema, std::unique_ptr* out); + static Status Open(const std::shared_ptr& schema, + std::unique_ptr* out); Status WriteRecordBatch(const RecordBatch& batch); Status Finish(std::string* result); @@ -61,11 +61,11 @@ class ARROW_EXPORT JsonReader { ~JsonReader(); static Status Open(MemoryPool* pool, const std::shared_ptr& data, - std::unique_ptr* reader); + std::unique_ptr* reader); // Use the default memory pool - static Status Open( - const std::shared_ptr& data, std::unique_ptr* reader); + static Status Open(const std::shared_ptr& data, + std::unique_ptr* reader); std::shared_ptr schema() const;