From e8f897310227a53d2a46c2aa071156fa66c575b1 Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sat, 18 Mar 2017 21:47:48 -0400 Subject: [PATCH 1/9] Split adapter.h/cc into reader.h/writer.h. Draft LargeRecordBatch type Change-Id: I33cfa0c572a74920666216351391e2c887ba45ca --- cpp/src/arrow/allocator-test.cc | 2 +- cpp/src/arrow/allocator.h | 4 +- cpp/src/arrow/ipc/CMakeLists.txt | 10 +- cpp/src/arrow/ipc/adapter.cc | 630 ------------------ cpp/src/arrow/ipc/adapter.h | 104 --- cpp/src/arrow/ipc/api.h | 1 - cpp/src/arrow/ipc/ipc-file-test.cc | 228 ------- ...adapter-test.cc => ipc-read-write-test.cc} | 204 +++++- cpp/src/arrow/ipc/metadata.cc | 4 +- cpp/src/arrow/ipc/metadata.h | 4 +- cpp/src/arrow/ipc/reader.cc | 114 +++- cpp/src/arrow/ipc/reader.h | 14 + cpp/src/arrow/ipc/writer.cc | 478 ++++++++++++- cpp/src/arrow/ipc/writer.h | 34 + format/Message.fbs | 16 + 15 files changed, 851 insertions(+), 996 deletions(-) delete mode 100644 cpp/src/arrow/ipc/adapter.cc delete mode 100644 cpp/src/arrow/ipc/adapter.h delete mode 100644 cpp/src/arrow/ipc/ipc-file-test.cc rename cpp/src/arrow/ipc/{ipc-adapter-test.cc => ipc-read-write-test.cc} (60%) diff --git a/cpp/src/arrow/allocator-test.cc b/cpp/src/arrow/allocator-test.cc index 0b242674bf1..a78d170ba12 100644 --- a/cpp/src/arrow/allocator-test.cc +++ b/cpp/src/arrow/allocator-test.cc @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -#include "gtest/gtest.h" #include "arrow/allocator.h" #include "arrow/test-util.h" +#include "gtest/gtest.h" namespace arrow { diff --git a/cpp/src/arrow/allocator.h b/cpp/src/arrow/allocator.h index c976ba96b8d..6a054aea228 100644 --- a/cpp/src/arrow/allocator.h +++ b/cpp/src/arrow/allocator.h @@ -18,11 +18,11 @@ #ifndef ARROW_ALLOCATOR_H #define ARROW_ALLOCATOR_H +#include "arrow/memory_pool.h" +#include "arrow/status.h" #include #include #include -#include "arrow/memory_pool.h" -#include "arrow/status.h" namespace arrow { diff --git a/cpp/src/arrow/ipc/CMakeLists.txt b/cpp/src/arrow/ipc/CMakeLists.txt index c73af63285b..403ac6d4c0f 100644 --- a/cpp/src/arrow/ipc/CMakeLists.txt +++ b/cpp/src/arrow/ipc/CMakeLists.txt @@ -29,7 +29,6 @@ set(ARROW_IPC_TEST_LINK_LIBS arrow_io_static) set(ARROW_IPC_SRCS - adapter.cc feather.cc json.cc json-internal.cc @@ -64,12 +63,8 @@ ADD_ARROW_TEST(feather-test) ARROW_TEST_LINK_LIBRARIES(feather-test ${ARROW_IPC_TEST_LINK_LIBS}) -ADD_ARROW_TEST(ipc-adapter-test) -ARROW_TEST_LINK_LIBRARIES(ipc-adapter-test - ${ARROW_IPC_TEST_LINK_LIBS}) - -ADD_ARROW_TEST(ipc-file-test) -ARROW_TEST_LINK_LIBRARIES(ipc-file-test +ADD_ARROW_TEST(ipc-read-write-test) +ARROW_TEST_LINK_LIBRARIES(ipc-read-write-test ${ARROW_IPC_TEST_LINK_LIBS}) ADD_ARROW_TEST(ipc-metadata-test) @@ -148,7 +143,6 @@ add_dependencies(arrow_ipc_objlib metadata_fbs) # Headers: top level install(FILES - adapter.h api.h feather.h json.h diff --git a/cpp/src/arrow/ipc/adapter.cc b/cpp/src/arrow/ipc/adapter.cc deleted file mode 100644 index db9f63ca18c..00000000000 --- a/cpp/src/arrow/ipc/adapter.cc +++ /dev/null @@ -1,630 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "arrow/ipc/adapter.h" - -#include -#include -#include -#include -#include -#include - -#include "arrow/array.h" -#include "arrow/buffer.h" -#include "arrow/io/interfaces.h" -#include "arrow/io/memory.h" -#include "arrow/ipc/Message_generated.h" -#include "arrow/ipc/metadata-internal.h" -#include "arrow/ipc/metadata.h" -#include "arrow/ipc/util.h" -#include "arrow/loader.h" -#include "arrow/memory_pool.h" -#include "arrow/schema.h" -#include "arrow/status.h" -#include "arrow/table.h" -#include "arrow/type.h" -#include "arrow/type_fwd.h" -#include "arrow/util/bit-util.h" -#include "arrow/util/logging.h" - -namespace arrow { - -namespace flatbuf = org::apache::arrow::flatbuf; - -namespace ipc { - -// ---------------------------------------------------------------------- -// Record batch write path - -class RecordBatchWriter : public ArrayVisitor { - public: - RecordBatchWriter( - MemoryPool* pool, int64_t buffer_start_offset, int max_recursion_depth) - : pool_(pool), - max_recursion_depth_(max_recursion_depth), - buffer_start_offset_(buffer_start_offset) { - DCHECK_GT(max_recursion_depth, 0); - } - - virtual ~RecordBatchWriter() = default; - - Status VisitArray(const Array& arr) { - if (max_recursion_depth_ <= 0) { - return Status::Invalid("Max recursion depth reached"); - } - - if (arr.length() > std::numeric_limits::max()) { - return Status::Invalid("Cannot write arrays larger than 2^31 - 1 in length"); - } - - // push back all common elements - field_nodes_.push_back(flatbuf::FieldNode( - static_cast(arr.length()), static_cast(arr.null_count()))); - if (arr.null_count() > 0) { - std::shared_ptr bitmap = arr.null_bitmap(); - - if (arr.offset() != 0) { - // With a sliced array / non-zero offset, we must copy the bitmap - RETURN_NOT_OK( - CopyBitmap(pool_, bitmap->data(), arr.offset(), arr.length(), &bitmap)); - } - - buffers_.push_back(bitmap); - } else { - // Push a dummy zero-length buffer, not to be copied - buffers_.push_back(std::make_shared(nullptr, 0)); - } - return arr.Accept(this); - } - - Status Assemble(const RecordBatch& batch, int64_t* body_length) { - if (field_nodes_.size() > 0) { - field_nodes_.clear(); - buffer_meta_.clear(); - buffers_.clear(); - } - - // Perform depth-first traversal of the row-batch - for (int i = 0; i < batch.num_columns(); ++i) { - RETURN_NOT_OK(VisitArray(*batch.column(i))); - } - - // The position for the start of a buffer relative to the passed frame of - // reference. May be 0 or some other position in an address space - int64_t offset = buffer_start_offset_; - - // Construct the buffer metadata for the record batch header - for (size_t i = 0; i < buffers_.size(); ++i) { - const Buffer* buffer = buffers_[i].get(); - int64_t size = 0; - int64_t padding = 0; - - // The buffer might be null if we are handling zero row lengths. - if (buffer) { - size = buffer->size(); - padding = BitUtil::RoundUpToMultipleOf64(size) - size; - } - - // TODO(wesm): We currently have no notion of shared memory page id's, - // but we've included it in the metadata IDL for when we have it in the - // future. Use page = -1 for now - // - // Note that page ids are a bespoke notion for Arrow and not a feature we - // are using from any OS-level shared memory. The thought is that systems - // may (in the future) associate integer page id's with physical memory - // pages (according to whatever is the desired shared memory mechanism) - buffer_meta_.push_back(flatbuf::Buffer(-1, offset, size + padding)); - offset += size + padding; - } - - *body_length = offset - buffer_start_offset_; - DCHECK(BitUtil::IsMultipleOf64(*body_length)); - - return Status::OK(); - } - - // Override this for writing dictionary metadata - virtual Status WriteMetadataMessage( - int32_t num_rows, int64_t body_length, std::shared_ptr* out) { - return WriteRecordBatchMessage( - num_rows, body_length, field_nodes_, buffer_meta_, out); - } - - Status WriteMetadata(int32_t num_rows, int64_t body_length, io::OutputStream* dst, - int32_t* metadata_length) { - // Now that we have computed the locations of all of the buffers in shared - // memory, the data header can be converted to a flatbuffer and written out - // - // Note: The memory written here is prefixed by the size of the flatbuffer - // itself as an int32_t. - std::shared_ptr metadata_fb; - RETURN_NOT_OK(WriteMetadataMessage(num_rows, body_length, &metadata_fb)); - - // Need to write 4 bytes (metadata size), the metadata, plus padding to - // end on an 8-byte offset - int64_t start_offset; - RETURN_NOT_OK(dst->Tell(&start_offset)); - - int32_t padded_metadata_length = static_cast(metadata_fb->size()) + 4; - const int32_t remainder = - (padded_metadata_length + static_cast(start_offset)) % 8; - if (remainder != 0) { padded_metadata_length += 8 - remainder; } - - // The returned metadata size includes the length prefix, the flatbuffer, - // plus padding - *metadata_length = padded_metadata_length; - - // Write the flatbuffer size prefix including padding - int32_t flatbuffer_size = padded_metadata_length - 4; - RETURN_NOT_OK( - dst->Write(reinterpret_cast(&flatbuffer_size), sizeof(int32_t))); - - // Write the flatbuffer - RETURN_NOT_OK(dst->Write(metadata_fb->data(), metadata_fb->size())); - - // Write any padding - int32_t padding = - padded_metadata_length - static_cast(metadata_fb->size()) - 4; - if (padding > 0) { RETURN_NOT_OK(dst->Write(kPaddingBytes, padding)); } - - return Status::OK(); - } - - Status Write(const RecordBatch& batch, io::OutputStream* dst, int32_t* metadata_length, - int64_t* body_length) { - RETURN_NOT_OK(Assemble(batch, body_length)); - -#ifndef NDEBUG - int64_t start_position, current_position; - RETURN_NOT_OK(dst->Tell(&start_position)); -#endif - - RETURN_NOT_OK(WriteMetadata( - static_cast(batch.num_rows()), *body_length, dst, metadata_length)); - -#ifndef NDEBUG - RETURN_NOT_OK(dst->Tell(¤t_position)); - DCHECK(BitUtil::IsMultipleOf8(current_position)); -#endif - - // Now write the buffers - for (size_t i = 0; i < buffers_.size(); ++i) { - const Buffer* buffer = buffers_[i].get(); - int64_t size = 0; - int64_t padding = 0; - - // The buffer might be null if we are handling zero row lengths. - if (buffer) { - size = buffer->size(); - padding = BitUtil::RoundUpToMultipleOf64(size) - size; - } - - if (size > 0) { RETURN_NOT_OK(dst->Write(buffer->data(), size)); } - - if (padding > 0) { RETURN_NOT_OK(dst->Write(kPaddingBytes, padding)); } - } - -#ifndef NDEBUG - RETURN_NOT_OK(dst->Tell(¤t_position)); - DCHECK(BitUtil::IsMultipleOf8(current_position)); -#endif - - return Status::OK(); - } - - Status GetTotalSize(const RecordBatch& batch, int64_t* size) { - // emulates the behavior of Write without actually writing - int32_t metadata_length = 0; - int64_t body_length = 0; - MockOutputStream dst; - RETURN_NOT_OK(Write(batch, &dst, &metadata_length, &body_length)); - *size = dst.GetExtentBytesWritten(); - return Status::OK(); - } - - protected: - template - Status VisitFixedWidth(const ArrayType& array) { - std::shared_ptr data_buffer = array.data(); - - if (array.offset() != 0) { - // Non-zero offset, slice the buffer - const auto& fw_type = static_cast(*array.type()); - const int type_width = fw_type.bit_width() / 8; - const int64_t byte_offset = array.offset() * type_width; - - // Send padding if it's available - const int64_t buffer_length = - std::min(BitUtil::RoundUpToMultipleOf64(array.length() * type_width), - data_buffer->size() - byte_offset); - data_buffer = SliceBuffer(data_buffer, byte_offset, buffer_length); - } - buffers_.push_back(data_buffer); - return Status::OK(); - } - - template - Status GetZeroBasedValueOffsets( - const ArrayType& array, std::shared_ptr* value_offsets) { - // Share slicing logic between ListArray and BinaryArray - - auto offsets = array.value_offsets(); - - if (array.offset() != 0) { - // If we have a non-zero offset, then the value offsets do not start at - // zero. We must a) create a new offsets array with shifted offsets and - // b) slice the values array accordingly - - std::shared_ptr shifted_offsets; - RETURN_NOT_OK(AllocateBuffer( - pool_, sizeof(int32_t) * (array.length() + 1), &shifted_offsets)); - - int32_t* dest_offsets = reinterpret_cast(shifted_offsets->mutable_data()); - const int32_t start_offset = array.value_offset(0); - - for (int i = 0; i < array.length(); ++i) { - dest_offsets[i] = array.value_offset(i) - start_offset; - } - // Final offset - dest_offsets[array.length()] = array.value_offset(array.length()) - start_offset; - offsets = shifted_offsets; - } - - *value_offsets = offsets; - return Status::OK(); - } - - Status VisitBinary(const BinaryArray& array) { - std::shared_ptr value_offsets; - RETURN_NOT_OK(GetZeroBasedValueOffsets(array, &value_offsets)); - auto data = array.data(); - - if (array.offset() != 0) { - // Slice the data buffer to include only the range we need now - data = SliceBuffer(data, array.value_offset(0), array.value_offset(array.length())); - } - - buffers_.push_back(value_offsets); - buffers_.push_back(data); - return Status::OK(); - } - - Status Visit(const FixedWidthBinaryArray& array) override { - auto data = array.data(); - int32_t width = array.byte_width(); - - if (array.offset() != 0) { - data = SliceBuffer(data, array.offset() * width, width * array.length()); - } - buffers_.push_back(data); - return Status::OK(); - } - - Status Visit(const BooleanArray& array) override { - buffers_.push_back(array.data()); - return Status::OK(); - } - -#define VISIT_FIXED_WIDTH(TYPE) \ - Status Visit(const TYPE& array) override { return VisitFixedWidth(array); } - - VISIT_FIXED_WIDTH(Int8Array); - VISIT_FIXED_WIDTH(Int16Array); - VISIT_FIXED_WIDTH(Int32Array); - VISIT_FIXED_WIDTH(Int64Array); - VISIT_FIXED_WIDTH(UInt8Array); - VISIT_FIXED_WIDTH(UInt16Array); - VISIT_FIXED_WIDTH(UInt32Array); - VISIT_FIXED_WIDTH(UInt64Array); - VISIT_FIXED_WIDTH(HalfFloatArray); - VISIT_FIXED_WIDTH(FloatArray); - VISIT_FIXED_WIDTH(DoubleArray); - VISIT_FIXED_WIDTH(DateArray); - VISIT_FIXED_WIDTH(Date32Array); - VISIT_FIXED_WIDTH(TimeArray); - VISIT_FIXED_WIDTH(TimestampArray); - -#undef VISIT_FIXED_WIDTH - - Status Visit(const StringArray& array) override { return VisitBinary(array); } - - Status Visit(const BinaryArray& array) override { return VisitBinary(array); } - - Status Visit(const ListArray& array) override { - std::shared_ptr value_offsets; - RETURN_NOT_OK(GetZeroBasedValueOffsets(array, &value_offsets)); - buffers_.push_back(value_offsets); - - --max_recursion_depth_; - std::shared_ptr values = array.values(); - - if (array.offset() != 0) { - // For non-zero offset, we slice the values array accordingly - const int32_t offset = array.value_offset(0); - const int32_t length = array.value_offset(array.length()) - offset; - values = values->Slice(offset, length); - } - RETURN_NOT_OK(VisitArray(*values)); - ++max_recursion_depth_; - return Status::OK(); - } - - Status Visit(const StructArray& array) override { - --max_recursion_depth_; - for (std::shared_ptr field : array.fields()) { - if (array.offset() != 0) { - // If offset is non-zero, slice the child array - field = field->Slice(array.offset(), array.length()); - } - RETURN_NOT_OK(VisitArray(*field)); - } - ++max_recursion_depth_; - return Status::OK(); - } - - Status Visit(const UnionArray& array) override { - auto type_ids = array.type_ids(); - if (array.offset() != 0) { - type_ids = SliceBuffer(type_ids, array.offset() * sizeof(UnionArray::type_id_t), - array.length() * sizeof(UnionArray::type_id_t)); - } - - buffers_.push_back(type_ids); - - --max_recursion_depth_; - if (array.mode() == UnionMode::DENSE) { - const auto& type = static_cast(*array.type()); - auto value_offsets = array.value_offsets(); - - // The Union type codes are not necessary 0-indexed - uint8_t max_code = 0; - for (uint8_t code : type.type_codes) { - if (code > max_code) { max_code = code; } - } - - // Allocate an array of child offsets. Set all to -1 to indicate that we - // haven't observed a first occurrence of a particular child yet - std::vector child_offsets(max_code + 1); - std::vector child_lengths(max_code + 1, 0); - - if (array.offset() != 0) { - // This is an unpleasant case. Because the offsets are different for - // each child array, when we have a sliced array, we need to "rebase" - // the value_offsets for each array - - const int32_t* unshifted_offsets = array.raw_value_offsets(); - const uint8_t* type_ids = array.raw_type_ids(); - - // Allocate the shifted offsets - std::shared_ptr shifted_offsets_buffer; - RETURN_NOT_OK(AllocateBuffer( - pool_, array.length() * sizeof(int32_t), &shifted_offsets_buffer)); - int32_t* shifted_offsets = - reinterpret_cast(shifted_offsets_buffer->mutable_data()); - - for (int64_t i = 0; i < array.length(); ++i) { - const uint8_t code = type_ids[i]; - int32_t shift = child_offsets[code]; - if (shift == -1) { child_offsets[code] = shift = unshifted_offsets[i]; } - shifted_offsets[i] = unshifted_offsets[i] - shift; - - // Update the child length to account for observed value - ++child_lengths[code]; - } - - value_offsets = shifted_offsets_buffer; - } - buffers_.push_back(value_offsets); - - // Visit children and slice accordingly - for (int i = 0; i < type.num_children(); ++i) { - std::shared_ptr child = array.child(i); - if (array.offset() != 0) { - const uint8_t code = type.type_codes[i]; - child = child->Slice(child_offsets[code], child_lengths[code]); - } - RETURN_NOT_OK(VisitArray(*child)); - } - } else { - for (std::shared_ptr child : array.children()) { - // Sparse union, slicing is simpler - if (array.offset() != 0) { - // If offset is non-zero, slice the child array - child = child->Slice(array.offset(), array.length()); - } - RETURN_NOT_OK(VisitArray(*child)); - } - } - ++max_recursion_depth_; - return Status::OK(); - } - - Status Visit(const DictionaryArray& array) override { - // Dictionary written out separately. Slice offset contained in the indices - return array.indices()->Accept(this); - } - - // In some cases, intermediate buffers may need to be allocated (with sliced arrays) - MemoryPool* pool_; - - std::vector field_nodes_; - std::vector buffer_meta_; - std::vector> buffers_; - - int64_t max_recursion_depth_; - int64_t buffer_start_offset_; -}; - -class DictionaryWriter : public RecordBatchWriter { - public: - using RecordBatchWriter::RecordBatchWriter; - - Status WriteMetadataMessage( - int32_t num_rows, int64_t body_length, std::shared_ptr* out) override { - return WriteDictionaryMessage( - dictionary_id_, num_rows, body_length, field_nodes_, buffer_meta_, out); - } - - Status Write(int64_t dictionary_id, const std::shared_ptr& dictionary, - io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length) { - dictionary_id_ = dictionary_id; - - // Make a dummy record batch. A bit tedious as we have to make a schema - std::vector> fields = { - arrow::field("dictionary", dictionary->type())}; - auto schema = std::make_shared(fields); - RecordBatch batch(schema, dictionary->length(), {dictionary}); - - return RecordBatchWriter::Write(batch, dst, metadata_length, body_length); - } - - private: - // TODO(wesm): Setting this in Write is a bit unclean, but it works - int64_t dictionary_id_; -}; - -Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, - io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, - MemoryPool* pool, int max_recursion_depth) { - RecordBatchWriter writer(pool, buffer_start_offset, max_recursion_depth); - return writer.Write(batch, dst, metadata_length, body_length); -} - -Status WriteDictionary(int64_t dictionary_id, const std::shared_ptr& dictionary, - int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, - int64_t* body_length, MemoryPool* pool) { - DictionaryWriter writer(pool, buffer_start_offset, kMaxNestingDepth); - return writer.Write(dictionary_id, dictionary, dst, metadata_length, body_length); -} - -Status GetRecordBatchSize(const RecordBatch& batch, int64_t* size) { - RecordBatchWriter writer(default_memory_pool(), 0, kMaxNestingDepth); - RETURN_NOT_OK(writer.GetTotalSize(batch, size)); - return Status::OK(); -} - -// ---------------------------------------------------------------------- -// Record batch read path - -class IpcComponentSource : public ArrayComponentSource { - public: - IpcComponentSource(const RecordBatchMetadata& metadata, io::RandomAccessFile* file) - : metadata_(metadata), file_(file) {} - - Status GetBuffer(int buffer_index, std::shared_ptr* out) override { - BufferMetadata buffer_meta = metadata_.buffer(buffer_index); - if (buffer_meta.length == 0) { - *out = nullptr; - return Status::OK(); - } else { - return file_->ReadAt(buffer_meta.offset, buffer_meta.length, out); - } - } - - Status GetFieldMetadata(int field_index, FieldMetadata* metadata) override { - // pop off a field - if (field_index >= metadata_.num_fields()) { - return Status::Invalid("Ran out of field metadata, likely malformed"); - } - *metadata = metadata_.field(field_index); - return Status::OK(); - } - - private: - const RecordBatchMetadata& metadata_; - io::RandomAccessFile* file_; -}; - -class RecordBatchReader { - public: - RecordBatchReader(const RecordBatchMetadata& metadata, - const std::shared_ptr& schema, int max_recursion_depth, - io::RandomAccessFile* file) - : metadata_(metadata), - schema_(schema), - max_recursion_depth_(max_recursion_depth), - file_(file) {} - - Status Read(std::shared_ptr* out) { - std::vector> arrays(schema_->num_fields()); - - IpcComponentSource source(metadata_, file_); - ArrayLoaderContext context; - context.source = &source; - context.field_index = 0; - context.buffer_index = 0; - context.max_recursion_depth = max_recursion_depth_; - - for (int i = 0; i < schema_->num_fields(); ++i) { - RETURN_NOT_OK(LoadArray(schema_->field(i)->type, &context, &arrays[i])); - } - - *out = std::make_shared(schema_, metadata_.length(), arrays); - return Status::OK(); - } - - private: - const RecordBatchMetadata& metadata_; - std::shared_ptr schema_; - int max_recursion_depth_; - io::RandomAccessFile* file_; -}; - -Status ReadRecordBatch(const RecordBatchMetadata& metadata, - const std::shared_ptr& schema, io::RandomAccessFile* file, - std::shared_ptr* out) { - return ReadRecordBatch(metadata, schema, kMaxNestingDepth, file, out); -} - -Status ReadRecordBatch(const RecordBatchMetadata& metadata, - const std::shared_ptr& schema, int max_recursion_depth, - io::RandomAccessFile* file, std::shared_ptr* out) { - RecordBatchReader reader(metadata, schema, max_recursion_depth, file); - return reader.Read(out); -} - -Status ReadDictionary(const DictionaryBatchMetadata& metadata, - const DictionaryTypeMap& dictionary_types, io::RandomAccessFile* file, - std::shared_ptr* out) { - int64_t id = metadata.id(); - auto it = dictionary_types.find(id); - if (it == dictionary_types.end()) { - std::stringstream ss; - ss << "Do not have type metadata for dictionary with id: " << id; - return Status::KeyError(ss.str()); - } - - std::vector> fields = {it->second}; - - // We need a schema for the record batch - auto dummy_schema = std::make_shared(fields); - - // The dictionary is embedded in a record batch with a single column - std::shared_ptr batch; - RETURN_NOT_OK(ReadRecordBatch(metadata.record_batch(), dummy_schema, file, &batch)); - - if (batch->num_columns() != 1) { - return Status::Invalid("Dictionary record batch must only contain one field"); - } - - *out = batch->column(0); - return Status::OK(); -} - -} // namespace ipc -} // namespace arrow diff --git a/cpp/src/arrow/ipc/adapter.h b/cpp/src/arrow/ipc/adapter.h deleted file mode 100644 index cea46860774..00000000000 --- a/cpp/src/arrow/ipc/adapter.h +++ /dev/null @@ -1,104 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// Public API for writing and accessing (with zero copy, if possible) Arrow -// IPC binary formatted data (e.g. in shared memory, or from some other IO source) - -#ifndef ARROW_IPC_ADAPTER_H -#define ARROW_IPC_ADAPTER_H - -#include -#include -#include - -#include "arrow/ipc/metadata.h" -#include "arrow/loader.h" -#include "arrow/util/visibility.h" - -namespace arrow { - -class Array; -class MemoryPool; -class RecordBatch; -class Schema; -class Status; - -namespace io { - -class RandomAccessFile; -class OutputStream; - -} // namespace io - -namespace ipc { - -// ---------------------------------------------------------------------- -// Write path - -// Write the RecordBatch (collection of equal-length Arrow arrays) to the -// output stream in a contiguous block. The record batch metadata is written as -// a flatbuffer (see format/Message.fbs -- the RecordBatch message type) -// prefixed by its size, followed by each of the memory buffers in the batch -// written end to end (with appropriate alignment and padding): -// -// -// -// Finally, the absolute offsets (relative to the start of the output stream) -// to the end of the body and end of the metadata / data header (suffixed by -// the header size) is returned in out-variables -// -// @param(in) buffer_start_offset: the start offset to use in the buffer metadata, -// default should be 0 -// -// @param(out) metadata_length: the size of the length-prefixed flatbuffer -// including padding to a 64-byte boundary -// -// @param(out) body_length: the size of the contiguous buffer block plus -// padding bytes -Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, - io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, - MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); - -// Write Array as a DictionaryBatch message -Status WriteDictionary(int64_t dictionary_id, const std::shared_ptr& dictionary, - int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, - int64_t* body_length, MemoryPool* pool); - -// Compute the precise number of bytes needed in a contiguous memory segment to -// write the record batch. This involves generating the complete serialized -// Flatbuffers metadata. -Status GetRecordBatchSize(const RecordBatch& batch, int64_t* size); - -// ---------------------------------------------------------------------- -// "Read" path; does not copy data if the input supports zero copy reads - -Status ReadRecordBatch(const RecordBatchMetadata& metadata, - const std::shared_ptr& schema, io::RandomAccessFile* file, - std::shared_ptr* out); - -Status ReadRecordBatch(const RecordBatchMetadata& metadata, - const std::shared_ptr& schema, int max_recursion_depth, - io::RandomAccessFile* file, std::shared_ptr* out); - -Status ReadDictionary(const DictionaryBatchMetadata& metadata, - const DictionaryTypeMap& dictionary_types, io::RandomAccessFile* file, - std::shared_ptr* out); - -} // namespace ipc -} // namespace arrow - -#endif // ARROW_IPC_MEMORY_H diff --git a/cpp/src/arrow/ipc/api.h b/cpp/src/arrow/ipc/api.h index ad7cd84e9f9..3f05e69d584 100644 --- a/cpp/src/arrow/ipc/api.h +++ b/cpp/src/arrow/ipc/api.h @@ -18,7 +18,6 @@ #ifndef ARROW_IPC_API_H #define ARROW_IPC_API_H -#include "arrow/ipc/adapter.h" #include "arrow/ipc/feather.h" #include "arrow/ipc/json.h" #include "arrow/ipc/metadata.h" diff --git a/cpp/src/arrow/ipc/ipc-file-test.cc b/cpp/src/arrow/ipc/ipc-file-test.cc deleted file mode 100644 index b45782220e4..00000000000 --- a/cpp/src/arrow/ipc/ipc-file-test.cc +++ /dev/null @@ -1,228 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include -#include -#include -#include -#include -#include - -#include "gtest/gtest.h" - -#include "arrow/array.h" -#include "arrow/io/memory.h" -#include "arrow/io/test-common.h" -#include "arrow/ipc/adapter.h" -#include "arrow/ipc/reader.h" -#include "arrow/ipc/test-common.h" -#include "arrow/ipc/util.h" -#include "arrow/ipc/writer.h" - -#include "arrow/buffer.h" -#include "arrow/memory_pool.h" -#include "arrow/status.h" -#include "arrow/test-util.h" -#include "arrow/util/bit-util.h" - -namespace arrow { -namespace ipc { - -void CompareBatch(const RecordBatch& left, const RecordBatch& right) { - if (!left.schema()->Equals(right.schema())) { - FAIL() << "Left schema: " << left.schema()->ToString() - << "\nRight schema: " << right.schema()->ToString(); - } - ASSERT_EQ(left.num_columns(), right.num_columns()) - << left.schema()->ToString() << " result: " << right.schema()->ToString(); - EXPECT_EQ(left.num_rows(), right.num_rows()); - for (int i = 0; i < left.num_columns(); ++i) { - EXPECT_TRUE(left.column(i)->Equals(right.column(i))) - << "Idx: " << i << " Name: " << left.column_name(i); - } -} - -using BatchVector = std::vector>; - -class TestFileFormat : public ::testing::TestWithParam { - public: - void SetUp() { - pool_ = default_memory_pool(); - buffer_ = std::make_shared(pool_); - sink_.reset(new io::BufferOutputStream(buffer_)); - } - void TearDown() {} - - Status RoundTripHelper(const BatchVector& in_batches, BatchVector* out_batches) { - // Write the file - std::shared_ptr writer; - RETURN_NOT_OK(FileWriter::Open(sink_.get(), in_batches[0]->schema(), &writer)); - - const int num_batches = static_cast(in_batches.size()); - - for (const auto& batch : in_batches) { - RETURN_NOT_OK(writer->WriteRecordBatch(*batch)); - } - RETURN_NOT_OK(writer->Close()); - RETURN_NOT_OK(sink_->Close()); - - // Current offset into stream is the end of the file - int64_t footer_offset; - RETURN_NOT_OK(sink_->Tell(&footer_offset)); - - // Open the file - auto buf_reader = std::make_shared(buffer_); - std::shared_ptr reader; - RETURN_NOT_OK(FileReader::Open(buf_reader, footer_offset, &reader)); - - EXPECT_EQ(num_batches, reader->num_record_batches()); - for (int i = 0; i < num_batches; ++i) { - std::shared_ptr chunk; - RETURN_NOT_OK(reader->GetRecordBatch(i, &chunk)); - out_batches->emplace_back(chunk); - } - - return Status::OK(); - } - - protected: - MemoryPool* pool_; - - std::unique_ptr sink_; - std::shared_ptr buffer_; -}; - -TEST_P(TestFileFormat, RoundTrip) { - std::shared_ptr batch1; - std::shared_ptr batch2; - ASSERT_OK((*GetParam())(&batch1)); // NOLINT clang-tidy gtest issue - ASSERT_OK((*GetParam())(&batch2)); // NOLINT clang-tidy gtest issue - - std::vector> in_batches = {batch1, batch2}; - std::vector> out_batches; - - ASSERT_OK(RoundTripHelper(in_batches, &out_batches)); - - // Compare batches - for (size_t i = 0; i < in_batches.size(); ++i) { - CompareBatch(*in_batches[i], *out_batches[i]); - } -} - -class TestStreamFormat : public ::testing::TestWithParam { - public: - void SetUp() { - pool_ = default_memory_pool(); - buffer_ = std::make_shared(pool_); - sink_.reset(new io::BufferOutputStream(buffer_)); - } - void TearDown() {} - - Status RoundTripHelper( - const RecordBatch& batch, std::vector>* out_batches) { - // Write the file - std::shared_ptr writer; - RETURN_NOT_OK(StreamWriter::Open(sink_.get(), batch.schema(), &writer)); - int num_batches = 5; - for (int i = 0; i < num_batches; ++i) { - RETURN_NOT_OK(writer->WriteRecordBatch(batch)); - } - RETURN_NOT_OK(writer->Close()); - RETURN_NOT_OK(sink_->Close()); - - // Open the file - auto buf_reader = std::make_shared(buffer_); - - std::shared_ptr reader; - RETURN_NOT_OK(StreamReader::Open(buf_reader, &reader)); - - std::shared_ptr chunk; - while (true) { - RETURN_NOT_OK(reader->GetNextRecordBatch(&chunk)); - if (chunk == nullptr) { break; } - out_batches->emplace_back(chunk); - } - return Status::OK(); - } - - protected: - MemoryPool* pool_; - - std::unique_ptr sink_; - std::shared_ptr buffer_; -}; - -TEST_P(TestStreamFormat, RoundTrip) { - std::shared_ptr batch; - ASSERT_OK((*GetParam())(&batch)); // NOLINT clang-tidy gtest issue - - std::vector> out_batches; - - ASSERT_OK(RoundTripHelper(*batch, &out_batches)); - - // Compare batches. Same - for (size_t i = 0; i < out_batches.size(); ++i) { - CompareBatch(*batch, *out_batches[i]); - } -} - -#define BATCH_CASES() \ - ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \ - &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch, \ - &MakeStruct, &MakeUnion, &MakeDictionary, &MakeDate, &MakeTimestamps, &MakeTimes, \ - &MakeFWBinary); - -INSTANTIATE_TEST_CASE_P(FileRoundTripTests, TestFileFormat, BATCH_CASES()); -INSTANTIATE_TEST_CASE_P(StreamRoundTripTests, TestStreamFormat, BATCH_CASES()); - -void CheckBatchDictionaries(const RecordBatch& batch) { - // Check that dictionaries that should be the same are the same - auto schema = batch.schema(); - - const auto& t0 = static_cast(*schema->field(0)->type); - const auto& t1 = static_cast(*schema->field(1)->type); - - ASSERT_EQ(t0.dictionary().get(), t1.dictionary().get()); - - // Same dictionary used for list values - const auto& t3 = static_cast(*schema->field(3)->type); - const auto& t3_value = static_cast(*t3.value_type()); - ASSERT_EQ(t0.dictionary().get(), t3_value.dictionary().get()); -} - -TEST_F(TestStreamFormat, DictionaryRoundTrip) { - std::shared_ptr batch; - ASSERT_OK(MakeDictionary(&batch)); - - std::vector> out_batches; - ASSERT_OK(RoundTripHelper(*batch, &out_batches)); - - CheckBatchDictionaries(*out_batches[0]); -} - -TEST_F(TestFileFormat, DictionaryRoundTrip) { - std::shared_ptr batch; - ASSERT_OK(MakeDictionary(&batch)); - - std::vector> out_batches; - ASSERT_OK(RoundTripHelper({batch}, &out_batches)); - - CheckBatchDictionaries(*out_batches[0]); -} - -} // namespace ipc -} // namespace arrow diff --git a/cpp/src/arrow/ipc/ipc-adapter-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc similarity index 60% rename from cpp/src/arrow/ipc/ipc-adapter-test.cc rename to cpp/src/arrow/ipc/ipc-read-write-test.cc index 638d98af824..d2cc7b1cf60 100644 --- a/cpp/src/arrow/ipc/ipc-adapter-test.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc @@ -24,10 +24,10 @@ #include "gtest/gtest.h" +#include "arrow/array.h" #include "arrow/io/memory.h" #include "arrow/io/test-common.h" -#include "arrow/ipc/adapter.h" -#include "arrow/ipc/metadata.h" +#include "arrow/ipc/api.h" #include "arrow/ipc/test-common.h" #include "arrow/ipc/util.h" @@ -41,6 +41,28 @@ namespace arrow { namespace ipc { +void CompareBatch(const RecordBatch& left, const RecordBatch& right) { + if (!left.schema()->Equals(right.schema())) { + FAIL() << "Left schema: " << left.schema()->ToString() + << "\nRight schema: " << right.schema()->ToString(); + } + ASSERT_EQ(left.num_columns(), right.num_columns()) + << left.schema()->ToString() << " result: " << right.schema()->ToString(); + EXPECT_EQ(left.num_rows(), right.num_rows()); + for (int i = 0; i < left.num_columns(); ++i) { + EXPECT_TRUE(left.column(i)->Equals(right.column(i))) + << "Idx: " << i << " Name: " << left.column_name(i); + } +} + +using BatchVector = std::vector>; + +#define BATCH_CASES() \ + ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \ + &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch, \ + &MakeStruct, &MakeUnion, &MakeDictionary, &MakeDate, &MakeTimestamps, &MakeTimes, \ + &MakeFWBinary); + class IpcTestFixture : public io::MemoryMapFixture { public: Status RoundTripHelper(const RecordBatch& batch, int memory_map_size, @@ -116,8 +138,8 @@ class TestWriteRecordBatch : public ::testing::Test, public IpcTestFixture { void TearDown() { io::MemoryMapFixture::TearDown(); } }; -class TestRecordBatchParam : public ::testing::TestWithParam, - public IpcTestFixture { +class TestIpcRoundTrip : public ::testing::TestWithParam, + public IpcTestFixture { public: void SetUp() { pool_ = default_memory_pool(); } void TearDown() { io::MemoryMapFixture::TearDown(); } @@ -125,14 +147,14 @@ class TestRecordBatchParam : public ::testing::TestWithParam, using IpcTestFixture::CheckRoundtrip; }; -TEST_P(TestRecordBatchParam, RoundTrip) { +TEST_P(TestIpcRoundTrip, RoundTrip) { std::shared_ptr batch; ASSERT_OK((*GetParam())(&batch)); // NOLINT clang-tidy gtest issue CheckRoundtrip(*batch, 1 << 20); } -TEST_P(TestRecordBatchParam, SliceRoundTrip) { +TEST_P(TestIpcRoundTrip, SliceRoundTrip) { std::shared_ptr batch; ASSERT_OK((*GetParam())(&batch)); // NOLINT clang-tidy gtest issue @@ -143,7 +165,7 @@ TEST_P(TestRecordBatchParam, SliceRoundTrip) { CheckRoundtrip(*sliced_batch, 1 << 20); } -TEST_P(TestRecordBatchParam, ZeroLengthArrays) { +TEST_P(TestIpcRoundTrip, ZeroLengthArrays) { std::shared_ptr batch; ASSERT_OK((*GetParam())(&batch)); // NOLINT clang-tidy gtest issue @@ -171,13 +193,6 @@ TEST_P(TestRecordBatchParam, ZeroLengthArrays) { CheckRoundtrip(bin_array2, 1 << 20); } -INSTANTIATE_TEST_CASE_P( - RoundTripTests, TestRecordBatchParam, - ::testing::Values(&MakeIntRecordBatch, &MakeStringTypesRecordBatch, - &MakeNonNullRecordBatch, &MakeZeroLengthRecordBatch, &MakeListRecordBatch, - &MakeDeeplyNestedList, &MakeStruct, &MakeUnion, &MakeDictionary, &MakeDate, - &MakeTimestamps, &MakeTimes, &MakeFWBinary)); - void TestGetRecordBatchSize(std::shared_ptr batch) { ipc::MockOutputStream mock; int32_t mock_metadata_length = -1; @@ -316,5 +331,166 @@ TEST_F(RecursionLimits, StressLimit) { ASSERT_TRUE(it_works); } +class TestFileFormat : public ::testing::TestWithParam { + public: + void SetUp() { + pool_ = default_memory_pool(); + buffer_ = std::make_shared(pool_); + sink_.reset(new io::BufferOutputStream(buffer_)); + } + void TearDown() {} + + Status RoundTripHelper(const BatchVector& in_batches, BatchVector* out_batches) { + // Write the file + std::shared_ptr writer; + RETURN_NOT_OK(FileWriter::Open(sink_.get(), in_batches[0]->schema(), &writer)); + + const int num_batches = static_cast(in_batches.size()); + + for (const auto& batch : in_batches) { + RETURN_NOT_OK(writer->WriteRecordBatch(*batch)); + } + RETURN_NOT_OK(writer->Close()); + RETURN_NOT_OK(sink_->Close()); + + // Current offset into stream is the end of the file + int64_t footer_offset; + RETURN_NOT_OK(sink_->Tell(&footer_offset)); + + // Open the file + auto buf_reader = std::make_shared(buffer_); + std::shared_ptr reader; + RETURN_NOT_OK(FileReader::Open(buf_reader, footer_offset, &reader)); + + EXPECT_EQ(num_batches, reader->num_record_batches()); + for (int i = 0; i < num_batches; ++i) { + std::shared_ptr chunk; + RETURN_NOT_OK(reader->GetRecordBatch(i, &chunk)); + out_batches->emplace_back(chunk); + } + + return Status::OK(); + } + + protected: + MemoryPool* pool_; + + std::unique_ptr sink_; + std::shared_ptr buffer_; +}; + +TEST_P(TestFileFormat, RoundTrip) { + std::shared_ptr batch1; + std::shared_ptr batch2; + ASSERT_OK((*GetParam())(&batch1)); // NOLINT clang-tidy gtest issue + ASSERT_OK((*GetParam())(&batch2)); // NOLINT clang-tidy gtest issue + + std::vector> in_batches = {batch1, batch2}; + std::vector> out_batches; + + ASSERT_OK(RoundTripHelper(in_batches, &out_batches)); + + // Compare batches + for (size_t i = 0; i < in_batches.size(); ++i) { + CompareBatch(*in_batches[i], *out_batches[i]); + } +} + +class TestStreamFormat : public ::testing::TestWithParam { + public: + void SetUp() { + pool_ = default_memory_pool(); + buffer_ = std::make_shared(pool_); + sink_.reset(new io::BufferOutputStream(buffer_)); + } + void TearDown() {} + + Status RoundTripHelper( + const RecordBatch& batch, std::vector>* out_batches) { + // Write the file + std::shared_ptr writer; + RETURN_NOT_OK(StreamWriter::Open(sink_.get(), batch.schema(), &writer)); + int num_batches = 5; + for (int i = 0; i < num_batches; ++i) { + RETURN_NOT_OK(writer->WriteRecordBatch(batch)); + } + RETURN_NOT_OK(writer->Close()); + RETURN_NOT_OK(sink_->Close()); + + // Open the file + auto buf_reader = std::make_shared(buffer_); + + std::shared_ptr reader; + RETURN_NOT_OK(StreamReader::Open(buf_reader, &reader)); + + std::shared_ptr chunk; + while (true) { + RETURN_NOT_OK(reader->GetNextRecordBatch(&chunk)); + if (chunk == nullptr) { break; } + out_batches->emplace_back(chunk); + } + return Status::OK(); + } + + protected: + MemoryPool* pool_; + + std::unique_ptr sink_; + std::shared_ptr buffer_; +}; + +TEST_P(TestStreamFormat, RoundTrip) { + std::shared_ptr batch; + ASSERT_OK((*GetParam())(&batch)); // NOLINT clang-tidy gtest issue + + std::vector> out_batches; + + ASSERT_OK(RoundTripHelper(*batch, &out_batches)); + + // Compare batches. Same + for (size_t i = 0; i < out_batches.size(); ++i) { + CompareBatch(*batch, *out_batches[i]); + } +} + +INSTANTIATE_TEST_CASE_P(GenericIpcRoundTripTests, TestIpcRoundTrip, BATCH_CASES()); +INSTANTIATE_TEST_CASE_P(FileRoundTripTests, TestFileFormat, BATCH_CASES()); +INSTANTIATE_TEST_CASE_P(StreamRoundTripTests, TestStreamFormat, BATCH_CASES()); + +void CheckBatchDictionaries(const RecordBatch& batch) { + // Check that dictionaries that should be the same are the same + auto schema = batch.schema(); + + const auto& t0 = static_cast(*schema->field(0)->type); + const auto& t1 = static_cast(*schema->field(1)->type); + + ASSERT_EQ(t0.dictionary().get(), t1.dictionary().get()); + + // Same dictionary used for list values + const auto& t3 = static_cast(*schema->field(3)->type); + const auto& t3_value = static_cast(*t3.value_type()); + ASSERT_EQ(t0.dictionary().get(), t3_value.dictionary().get()); +} + +TEST_F(TestStreamFormat, DictionaryRoundTrip) { + std::shared_ptr batch; + ASSERT_OK(MakeDictionary(&batch)); + + std::vector> out_batches; + ASSERT_OK(RoundTripHelper(*batch, &out_batches)); + + CheckBatchDictionaries(*out_batches[0]); +} + +TEST_F(TestFileFormat, DictionaryRoundTrip) { + std::shared_ptr batch; + ASSERT_OK(MakeDictionary(&batch)); + + std::vector> out_batches; + ASSERT_OK(RoundTripHelper({batch}, &out_batches)); + + CheckBatchDictionaries(*out_batches[0]); +} + } // namespace ipc } // namespace arrow diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc index 71bc5c9eb32..2b42d18713b 100644 --- a/cpp/src/arrow/ipc/metadata.cc +++ b/cpp/src/arrow/ipc/metadata.cc @@ -358,8 +358,8 @@ const RecordBatchMetadata& DictionaryBatchMetadata::record_batch() const { // ---------------------------------------------------------------------- // Conveniences -Status ReadMessage(int64_t offset, int32_t metadata_length, - io::RandomAccessFile* file, std::shared_ptr* message) { +Status ReadMessage(int64_t offset, int32_t metadata_length, io::RandomAccessFile* file, + std::shared_ptr* message) { std::shared_ptr buffer; RETURN_NOT_OK(file->ReadAt(offset, metadata_length, &buffer)); diff --git a/cpp/src/arrow/ipc/metadata.h b/cpp/src/arrow/ipc/metadata.h index 4eb0186d3a4..a378af7b9c5 100644 --- a/cpp/src/arrow/ipc/metadata.h +++ b/cpp/src/arrow/ipc/metadata.h @@ -218,8 +218,8 @@ class ARROW_EXPORT Message { /// \param[in] file the seekable file interface to read from /// \param[out] message the message read /// \return Status success or failure -Status ReadMessage(int64_t offset, int32_t metadata_length, - io::RandomAccessFile* file, std::shared_ptr* message); +Status ReadMessage(int64_t offset, int32_t metadata_length, io::RandomAccessFile* file, + std::shared_ptr* message); } // namespace ipc } // namespace arrow diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc index 95753643c65..813b82a7dbe 100644 --- a/cpp/src/arrow/ipc/reader.cc +++ b/cpp/src/arrow/ipc/reader.cc @@ -26,16 +26,125 @@ #include "arrow/buffer.h" #include "arrow/io/interfaces.h" #include "arrow/io/memory.h" -#include "arrow/ipc/adapter.h" #include "arrow/ipc/metadata-internal.h" #include "arrow/ipc/metadata.h" #include "arrow/ipc/util.h" +#include "arrow/schema.h" #include "arrow/status.h" +#include "arrow/table.h" #include "arrow/util/logging.h" namespace arrow { namespace ipc { +// ---------------------------------------------------------------------- +// Record batch read path + +class IpcComponentSource : public ArrayComponentSource { + public: + IpcComponentSource(const RecordBatchMetadata& metadata, io::RandomAccessFile* file) + : metadata_(metadata), file_(file) {} + + Status GetBuffer(int buffer_index, std::shared_ptr* out) override { + BufferMetadata buffer_meta = metadata_.buffer(buffer_index); + if (buffer_meta.length == 0) { + *out = nullptr; + return Status::OK(); + } else { + return file_->ReadAt(buffer_meta.offset, buffer_meta.length, out); + } + } + + Status GetFieldMetadata(int field_index, FieldMetadata* metadata) override { + // pop off a field + if (field_index >= metadata_.num_fields()) { + return Status::Invalid("Ran out of field metadata, likely malformed"); + } + *metadata = metadata_.field(field_index); + return Status::OK(); + } + + private: + const RecordBatchMetadata& metadata_; + io::RandomAccessFile* file_; +}; + +class RecordBatchReader { + public: + RecordBatchReader(const RecordBatchMetadata& metadata, + const std::shared_ptr& schema, int max_recursion_depth, + io::RandomAccessFile* file) + : metadata_(metadata), + schema_(schema), + max_recursion_depth_(max_recursion_depth), + file_(file) {} + + Status Read(std::shared_ptr* out) { + std::vector> arrays(schema_->num_fields()); + + IpcComponentSource source(metadata_, file_); + ArrayLoaderContext context; + context.source = &source; + context.field_index = 0; + context.buffer_index = 0; + context.max_recursion_depth = max_recursion_depth_; + + for (int i = 0; i < schema_->num_fields(); ++i) { + RETURN_NOT_OK(LoadArray(schema_->field(i)->type, &context, &arrays[i])); + } + + *out = std::make_shared(schema_, metadata_.length(), arrays); + return Status::OK(); + } + + private: + const RecordBatchMetadata& metadata_; + std::shared_ptr schema_; + int max_recursion_depth_; + io::RandomAccessFile* file_; +}; + +Status ReadRecordBatch(const RecordBatchMetadata& metadata, + const std::shared_ptr& schema, io::RandomAccessFile* file, + std::shared_ptr* out) { + return ReadRecordBatch(metadata, schema, kMaxNestingDepth, file, out); +} + +Status ReadRecordBatch(const RecordBatchMetadata& metadata, + const std::shared_ptr& schema, int max_recursion_depth, + io::RandomAccessFile* file, std::shared_ptr* out) { + RecordBatchReader reader(metadata, schema, max_recursion_depth, file); + return reader.Read(out); +} + +Status ReadDictionary(const DictionaryBatchMetadata& metadata, + const DictionaryTypeMap& dictionary_types, io::RandomAccessFile* file, + std::shared_ptr* out) { + int64_t id = metadata.id(); + auto it = dictionary_types.find(id); + if (it == dictionary_types.end()) { + std::stringstream ss; + ss << "Do not have type metadata for dictionary with id: " << id; + return Status::KeyError(ss.str()); + } + + std::vector> fields = {it->second}; + + // We need a schema for the record batch + auto dummy_schema = std::make_shared(fields); + + // The dictionary is embedded in a record batch with a single column + std::shared_ptr batch; + RETURN_NOT_OK(ReadRecordBatch(metadata.record_batch(), dummy_schema, file, &batch)); + + if (batch->num_columns() != 1) { + return Status::Invalid("Dictionary record batch must only contain one field"); + } + + *out = batch->column(0); + return Status::OK(); +} + // ---------------------------------------------------------------------- // StreamReader implementation @@ -307,8 +416,7 @@ class FileReader::FileReaderImpl { return schema_metadata_->GetSchema(*dictionary_memo_, &schema_); } - Status Open( - const std::shared_ptr& file, int64_t footer_offset) { + Status Open(const std::shared_ptr& file, int64_t footer_offset) { file_ = file; footer_offset_ = footer_offset; RETURN_NOT_OK(ReadFooter()); diff --git a/cpp/src/arrow/ipc/reader.h b/cpp/src/arrow/ipc/reader.h index ca91765edba..43ada5a7877 100644 --- a/cpp/src/arrow/ipc/reader.h +++ b/cpp/src/arrow/ipc/reader.h @@ -43,6 +43,20 @@ class RandomAccessFile; namespace ipc { +// Generic read functionsh; does not copy data if the input supports zero copy reads + +Status ReadRecordBatch(const RecordBatchMetadata& metadata, + const std::shared_ptr& schema, io::RandomAccessFile* file, + std::shared_ptr* out); + +Status ReadRecordBatch(const RecordBatchMetadata& metadata, + const std::shared_ptr& schema, int max_recursion_depth, + io::RandomAccessFile* file, std::shared_ptr* out); + +Status ReadDictionary(const DictionaryBatchMetadata& metadata, + const DictionaryTypeMap& dictionary_types, io::RandomAccessFile* file, + std::shared_ptr* out); + class ARROW_EXPORT StreamReader { public: ~StreamReader(); diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index 58402b58840..869f46181b5 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -22,10 +22,11 @@ #include #include +#include "arrow/array.h" #include "arrow/buffer.h" #include "arrow/io/interfaces.h" #include "arrow/io/memory.h" -#include "arrow/ipc/adapter.h" +#include "arrow/ipc/Message_generated.h" #include "arrow/ipc/metadata-internal.h" #include "arrow/ipc/metadata.h" #include "arrow/ipc/util.h" @@ -33,11 +34,486 @@ #include "arrow/schema.h" #include "arrow/status.h" #include "arrow/table.h" +#include "arrow/type.h" +#include "arrow/util/bit-util.h" #include "arrow/util/logging.h" namespace arrow { + +namespace flatbuf = org::apache::arrow::flatbuf; + namespace ipc { +// ---------------------------------------------------------------------- +// Record batch write path + +class RecordBatchWriter : public ArrayVisitor { + public: + RecordBatchWriter( + MemoryPool* pool, int64_t buffer_start_offset, int max_recursion_depth) + : pool_(pool), + max_recursion_depth_(max_recursion_depth), + buffer_start_offset_(buffer_start_offset) { + DCHECK_GT(max_recursion_depth, 0); + } + + virtual ~RecordBatchWriter() = default; + + Status VisitArray(const Array& arr) { + if (max_recursion_depth_ <= 0) { + return Status::Invalid("Max recursion depth reached"); + } + + if (arr.length() > std::numeric_limits::max()) { + return Status::Invalid("Cannot write arrays larger than 2^31 - 1 in length"); + } + + // push back all common elements + field_nodes_.push_back(flatbuf::FieldNode( + static_cast(arr.length()), static_cast(arr.null_count()))); + if (arr.null_count() > 0) { + std::shared_ptr bitmap = arr.null_bitmap(); + + if (arr.offset() != 0) { + // With a sliced array / non-zero offset, we must copy the bitmap + RETURN_NOT_OK( + CopyBitmap(pool_, bitmap->data(), arr.offset(), arr.length(), &bitmap)); + } + + buffers_.push_back(bitmap); + } else { + // Push a dummy zero-length buffer, not to be copied + buffers_.push_back(std::make_shared(nullptr, 0)); + } + return arr.Accept(this); + } + + Status Assemble(const RecordBatch& batch, int64_t* body_length) { + if (field_nodes_.size() > 0) { + field_nodes_.clear(); + buffer_meta_.clear(); + buffers_.clear(); + } + + // Perform depth-first traversal of the row-batch + for (int i = 0; i < batch.num_columns(); ++i) { + RETURN_NOT_OK(VisitArray(*batch.column(i))); + } + + // The position for the start of a buffer relative to the passed frame of + // reference. May be 0 or some other position in an address space + int64_t offset = buffer_start_offset_; + + // Construct the buffer metadata for the record batch header + for (size_t i = 0; i < buffers_.size(); ++i) { + const Buffer* buffer = buffers_[i].get(); + int64_t size = 0; + int64_t padding = 0; + + // The buffer might be null if we are handling zero row lengths. + if (buffer) { + size = buffer->size(); + padding = BitUtil::RoundUpToMultipleOf64(size) - size; + } + + // TODO(wesm): We currently have no notion of shared memory page id's, + // but we've included it in the metadata IDL for when we have it in the + // future. Use page = -1 for now + // + // Note that page ids are a bespoke notion for Arrow and not a feature we + // are using from any OS-level shared memory. The thought is that systems + // may (in the future) associate integer page id's with physical memory + // pages (according to whatever is the desired shared memory mechanism) + buffer_meta_.push_back(flatbuf::Buffer(-1, offset, size + padding)); + offset += size + padding; + } + + *body_length = offset - buffer_start_offset_; + DCHECK(BitUtil::IsMultipleOf64(*body_length)); + + return Status::OK(); + } + + // Override this for writing dictionary metadata + virtual Status WriteMetadataMessage( + int32_t num_rows, int64_t body_length, std::shared_ptr* out) { + return WriteRecordBatchMessage( + num_rows, body_length, field_nodes_, buffer_meta_, out); + } + + Status WriteMetadata(int32_t num_rows, int64_t body_length, io::OutputStream* dst, + int32_t* metadata_length) { + // Now that we have computed the locations of all of the buffers in shared + // memory, the data header can be converted to a flatbuffer and written out + // + // Note: The memory written here is prefixed by the size of the flatbuffer + // itself as an int32_t. + std::shared_ptr metadata_fb; + RETURN_NOT_OK(WriteMetadataMessage(num_rows, body_length, &metadata_fb)); + + // Need to write 4 bytes (metadata size), the metadata, plus padding to + // end on an 8-byte offset + int64_t start_offset; + RETURN_NOT_OK(dst->Tell(&start_offset)); + + int32_t padded_metadata_length = static_cast(metadata_fb->size()) + 4; + const int32_t remainder = + (padded_metadata_length + static_cast(start_offset)) % 8; + if (remainder != 0) { padded_metadata_length += 8 - remainder; } + + // The returned metadata size includes the length prefix, the flatbuffer, + // plus padding + *metadata_length = padded_metadata_length; + + // Write the flatbuffer size prefix including padding + int32_t flatbuffer_size = padded_metadata_length - 4; + RETURN_NOT_OK( + dst->Write(reinterpret_cast(&flatbuffer_size), sizeof(int32_t))); + + // Write the flatbuffer + RETURN_NOT_OK(dst->Write(metadata_fb->data(), metadata_fb->size())); + + // Write any padding + int32_t padding = + padded_metadata_length - static_cast(metadata_fb->size()) - 4; + if (padding > 0) { RETURN_NOT_OK(dst->Write(kPaddingBytes, padding)); } + + return Status::OK(); + } + + Status Write(const RecordBatch& batch, io::OutputStream* dst, int32_t* metadata_length, + int64_t* body_length) { + RETURN_NOT_OK(Assemble(batch, body_length)); + +#ifndef NDEBUG + int64_t start_position, current_position; + RETURN_NOT_OK(dst->Tell(&start_position)); +#endif + + RETURN_NOT_OK(WriteMetadata( + static_cast(batch.num_rows()), *body_length, dst, metadata_length)); + +#ifndef NDEBUG + RETURN_NOT_OK(dst->Tell(¤t_position)); + DCHECK(BitUtil::IsMultipleOf8(current_position)); +#endif + + // Now write the buffers + for (size_t i = 0; i < buffers_.size(); ++i) { + const Buffer* buffer = buffers_[i].get(); + int64_t size = 0; + int64_t padding = 0; + + // The buffer might be null if we are handling zero row lengths. + if (buffer) { + size = buffer->size(); + padding = BitUtil::RoundUpToMultipleOf64(size) - size; + } + + if (size > 0) { RETURN_NOT_OK(dst->Write(buffer->data(), size)); } + + if (padding > 0) { RETURN_NOT_OK(dst->Write(kPaddingBytes, padding)); } + } + +#ifndef NDEBUG + RETURN_NOT_OK(dst->Tell(¤t_position)); + DCHECK(BitUtil::IsMultipleOf8(current_position)); +#endif + + return Status::OK(); + } + + Status GetTotalSize(const RecordBatch& batch, int64_t* size) { + // emulates the behavior of Write without actually writing + int32_t metadata_length = 0; + int64_t body_length = 0; + MockOutputStream dst; + RETURN_NOT_OK(Write(batch, &dst, &metadata_length, &body_length)); + *size = dst.GetExtentBytesWritten(); + return Status::OK(); + } + + protected: + template + Status VisitFixedWidth(const ArrayType& array) { + std::shared_ptr data_buffer = array.data(); + + if (array.offset() != 0) { + // Non-zero offset, slice the buffer + const auto& fw_type = static_cast(*array.type()); + const int type_width = fw_type.bit_width() / 8; + const int64_t byte_offset = array.offset() * type_width; + + // Send padding if it's available + const int64_t buffer_length = + std::min(BitUtil::RoundUpToMultipleOf64(array.length() * type_width), + data_buffer->size() - byte_offset); + data_buffer = SliceBuffer(data_buffer, byte_offset, buffer_length); + } + buffers_.push_back(data_buffer); + return Status::OK(); + } + + template + Status GetZeroBasedValueOffsets( + const ArrayType& array, std::shared_ptr* value_offsets) { + // Share slicing logic between ListArray and BinaryArray + + auto offsets = array.value_offsets(); + + if (array.offset() != 0) { + // If we have a non-zero offset, then the value offsets do not start at + // zero. We must a) create a new offsets array with shifted offsets and + // b) slice the values array accordingly + + std::shared_ptr shifted_offsets; + RETURN_NOT_OK(AllocateBuffer( + pool_, sizeof(int32_t) * (array.length() + 1), &shifted_offsets)); + + int32_t* dest_offsets = reinterpret_cast(shifted_offsets->mutable_data()); + const int32_t start_offset = array.value_offset(0); + + for (int i = 0; i < array.length(); ++i) { + dest_offsets[i] = array.value_offset(i) - start_offset; + } + // Final offset + dest_offsets[array.length()] = array.value_offset(array.length()) - start_offset; + offsets = shifted_offsets; + } + + *value_offsets = offsets; + return Status::OK(); + } + + Status VisitBinary(const BinaryArray& array) { + std::shared_ptr value_offsets; + RETURN_NOT_OK(GetZeroBasedValueOffsets(array, &value_offsets)); + auto data = array.data(); + + if (array.offset() != 0) { + // Slice the data buffer to include only the range we need now + data = SliceBuffer(data, array.value_offset(0), array.value_offset(array.length())); + } + + buffers_.push_back(value_offsets); + buffers_.push_back(data); + return Status::OK(); + } + + Status Visit(const FixedWidthBinaryArray& array) override { + auto data = array.data(); + int32_t width = array.byte_width(); + + if (array.offset() != 0) { + data = SliceBuffer(data, array.offset() * width, width * array.length()); + } + buffers_.push_back(data); + return Status::OK(); + } + + Status Visit(const BooleanArray& array) override { + buffers_.push_back(array.data()); + return Status::OK(); + } + +#define VISIT_FIXED_WIDTH(TYPE) \ + Status Visit(const TYPE& array) override { return VisitFixedWidth(array); } + + VISIT_FIXED_WIDTH(Int8Array); + VISIT_FIXED_WIDTH(Int16Array); + VISIT_FIXED_WIDTH(Int32Array); + VISIT_FIXED_WIDTH(Int64Array); + VISIT_FIXED_WIDTH(UInt8Array); + VISIT_FIXED_WIDTH(UInt16Array); + VISIT_FIXED_WIDTH(UInt32Array); + VISIT_FIXED_WIDTH(UInt64Array); + VISIT_FIXED_WIDTH(HalfFloatArray); + VISIT_FIXED_WIDTH(FloatArray); + VISIT_FIXED_WIDTH(DoubleArray); + VISIT_FIXED_WIDTH(DateArray); + VISIT_FIXED_WIDTH(Date32Array); + VISIT_FIXED_WIDTH(TimeArray); + VISIT_FIXED_WIDTH(TimestampArray); + +#undef VISIT_FIXED_WIDTH + + Status Visit(const StringArray& array) override { return VisitBinary(array); } + + Status Visit(const BinaryArray& array) override { return VisitBinary(array); } + + Status Visit(const ListArray& array) override { + std::shared_ptr value_offsets; + RETURN_NOT_OK(GetZeroBasedValueOffsets(array, &value_offsets)); + buffers_.push_back(value_offsets); + + --max_recursion_depth_; + std::shared_ptr values = array.values(); + + if (array.offset() != 0) { + // For non-zero offset, we slice the values array accordingly + const int32_t offset = array.value_offset(0); + const int32_t length = array.value_offset(array.length()) - offset; + values = values->Slice(offset, length); + } + RETURN_NOT_OK(VisitArray(*values)); + ++max_recursion_depth_; + return Status::OK(); + } + + Status Visit(const StructArray& array) override { + --max_recursion_depth_; + for (std::shared_ptr field : array.fields()) { + if (array.offset() != 0) { + // If offset is non-zero, slice the child array + field = field->Slice(array.offset(), array.length()); + } + RETURN_NOT_OK(VisitArray(*field)); + } + ++max_recursion_depth_; + return Status::OK(); + } + + Status Visit(const UnionArray& array) override { + auto type_ids = array.type_ids(); + if (array.offset() != 0) { + type_ids = SliceBuffer(type_ids, array.offset() * sizeof(UnionArray::type_id_t), + array.length() * sizeof(UnionArray::type_id_t)); + } + + buffers_.push_back(type_ids); + + --max_recursion_depth_; + if (array.mode() == UnionMode::DENSE) { + const auto& type = static_cast(*array.type()); + auto value_offsets = array.value_offsets(); + + // The Union type codes are not necessary 0-indexed + uint8_t max_code = 0; + for (uint8_t code : type.type_codes) { + if (code > max_code) { max_code = code; } + } + + // Allocate an array of child offsets. Set all to -1 to indicate that we + // haven't observed a first occurrence of a particular child yet + std::vector child_offsets(max_code + 1); + std::vector child_lengths(max_code + 1, 0); + + if (array.offset() != 0) { + // This is an unpleasant case. Because the offsets are different for + // each child array, when we have a sliced array, we need to "rebase" + // the value_offsets for each array + + const int32_t* unshifted_offsets = array.raw_value_offsets(); + const uint8_t* type_ids = array.raw_type_ids(); + + // Allocate the shifted offsets + std::shared_ptr shifted_offsets_buffer; + RETURN_NOT_OK(AllocateBuffer( + pool_, array.length() * sizeof(int32_t), &shifted_offsets_buffer)); + int32_t* shifted_offsets = + reinterpret_cast(shifted_offsets_buffer->mutable_data()); + + for (int64_t i = 0; i < array.length(); ++i) { + const uint8_t code = type_ids[i]; + int32_t shift = child_offsets[code]; + if (shift == -1) { child_offsets[code] = shift = unshifted_offsets[i]; } + shifted_offsets[i] = unshifted_offsets[i] - shift; + + // Update the child length to account for observed value + ++child_lengths[code]; + } + + value_offsets = shifted_offsets_buffer; + } + buffers_.push_back(value_offsets); + + // Visit children and slice accordingly + for (int i = 0; i < type.num_children(); ++i) { + std::shared_ptr child = array.child(i); + if (array.offset() != 0) { + const uint8_t code = type.type_codes[i]; + child = child->Slice(child_offsets[code], child_lengths[code]); + } + RETURN_NOT_OK(VisitArray(*child)); + } + } else { + for (std::shared_ptr child : array.children()) { + // Sparse union, slicing is simpler + if (array.offset() != 0) { + // If offset is non-zero, slice the child array + child = child->Slice(array.offset(), array.length()); + } + RETURN_NOT_OK(VisitArray(*child)); + } + } + ++max_recursion_depth_; + return Status::OK(); + } + + Status Visit(const DictionaryArray& array) override { + // Dictionary written out separately. Slice offset contained in the indices + return array.indices()->Accept(this); + } + + // In some cases, intermediate buffers may need to be allocated (with sliced arrays) + MemoryPool* pool_; + + std::vector field_nodes_; + std::vector buffer_meta_; + std::vector> buffers_; + + int64_t max_recursion_depth_; + int64_t buffer_start_offset_; +}; + +class DictionaryWriter : public RecordBatchWriter { + public: + using RecordBatchWriter::RecordBatchWriter; + + Status WriteMetadataMessage( + int32_t num_rows, int64_t body_length, std::shared_ptr* out) override { + return WriteDictionaryMessage( + dictionary_id_, num_rows, body_length, field_nodes_, buffer_meta_, out); + } + + Status Write(int64_t dictionary_id, const std::shared_ptr& dictionary, + io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length) { + dictionary_id_ = dictionary_id; + + // Make a dummy record batch. A bit tedious as we have to make a schema + std::vector> fields = { + arrow::field("dictionary", dictionary->type())}; + auto schema = std::make_shared(fields); + RecordBatch batch(schema, dictionary->length(), {dictionary}); + + return RecordBatchWriter::Write(batch, dst, metadata_length, body_length); + } + + private: + // TODO(wesm): Setting this in Write is a bit unclean, but it works + int64_t dictionary_id_; +}; + +Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, + io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, + MemoryPool* pool, int max_recursion_depth) { + RecordBatchWriter writer(pool, buffer_start_offset, max_recursion_depth); + return writer.Write(batch, dst, metadata_length, body_length); +} + +Status WriteDictionary(int64_t dictionary_id, const std::shared_ptr& dictionary, + int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, + int64_t* body_length, MemoryPool* pool) { + DictionaryWriter writer(pool, buffer_start_offset, kMaxNestingDepth); + return writer.Write(dictionary_id, dictionary, dst, metadata_length, body_length); +} + +Status GetRecordBatchSize(const RecordBatch& batch, int64_t* size) { + RecordBatchWriter writer(default_memory_pool(), 0, kMaxNestingDepth); + RETURN_NOT_OK(writer.GetTotalSize(batch, size)); + return Status::OK(); +} + // ---------------------------------------------------------------------- // Stream writer implementation diff --git a/cpp/src/arrow/ipc/writer.h b/cpp/src/arrow/ipc/writer.h index 7aff71e18e4..c1d70b5dca4 100644 --- a/cpp/src/arrow/ipc/writer.h +++ b/cpp/src/arrow/ipc/writer.h @@ -45,6 +45,40 @@ class OutputStream; namespace ipc { +// Write the RecordBatch (collection of equal-length Arrow arrays) to the +// output stream in a contiguous block. The record batch metadata is written as +// a flatbuffer (see format/Message.fbs -- the RecordBatch message type) +// prefixed by its size, followed by each of the memory buffers in the batch +// written end to end (with appropriate alignment and padding): +// +// +// +// Finally, the absolute offsets (relative to the start of the output stream) +// to the end of the body and end of the metadata / data header (suffixed by +// the header size) is returned in out-variables +// +// @param(in) buffer_start_offset: the start offset to use in the buffer metadata, +// default should be 0 +// +// @param(out) metadata_length: the size of the length-prefixed flatbuffer +// including padding to a 64-byte boundary +// +// @param(out) body_length: the size of the contiguous buffer block plus +// padding bytes +Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, + io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, + MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); + +// Write Array as a DictionaryBatch message +Status WriteDictionary(int64_t dictionary_id, const std::shared_ptr& dictionary, + int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, + int64_t* body_length, MemoryPool* pool); + +// Compute the precise number of bytes needed in a contiguous memory segment to +// write the record batch. This involves generating the complete serialized +// Flatbuffers metadata. +Status GetRecordBatchSize(const RecordBatch& batch, int64_t* size); + class ARROW_EXPORT StreamWriter { public: virtual ~StreamWriter() = default; diff --git a/format/Message.fbs b/format/Message.fbs index f2d5eba75e6..c54218edc49 100644 --- a/format/Message.fbs +++ b/format/Message.fbs @@ -302,6 +302,22 @@ table RecordBatch { buffers: [Buffer]; } +/// ---------------------------------------------------------------------- +/// EXPERIMENTAL: A RecordBatch type that supports data with more than 2^31 - 1 +/// elements. Arrow implementations do not need to implement this type to be +/// compliant + +struct LargeFieldNode { + length: int; + null_count: int; +} + +table LargeRecordBatch { + length: long; + nodes: [LargeFieldNode]; + buffers: [Buffer]; +} + /// ---------------------------------------------------------------------- /// For sending dictionary encoding information. Any Field can be /// dictionary-encoded, but in this case none of its children may be From 0f2722cc86d590f87866f67cfae8e4e049b33717 Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 00:18:30 -0400 Subject: [PATCH 2/9] Consolidate metadata-internal.h into metadata.h. Use own Arrow structs for IPC metadata and convert to flatbuffers later Change-Id: Ia464c912d41afd4f68d28e8a58cf9fc8ea5f8797 --- cpp/src/arrow/ipc/CMakeLists.txt | 1 - cpp/src/arrow/ipc/ipc-metadata-test.cc | 1 - cpp/src/arrow/ipc/metadata-internal.cc | 597 ----------------------- cpp/src/arrow/ipc/metadata-internal.h | 83 ---- cpp/src/arrow/ipc/metadata.cc | 625 ++++++++++++++++++++++++- cpp/src/arrow/ipc/metadata.h | 22 + cpp/src/arrow/ipc/reader.cc | 6 +- cpp/src/arrow/ipc/writer.cc | 52 +- cpp/src/arrow/ipc/writer.h | 10 +- cpp/src/arrow/loader.h | 25 + cpp/src/arrow/type.h | 1 + 11 files changed, 691 insertions(+), 732 deletions(-) delete mode 100644 cpp/src/arrow/ipc/metadata-internal.cc delete mode 100644 cpp/src/arrow/ipc/metadata-internal.h diff --git a/cpp/src/arrow/ipc/CMakeLists.txt b/cpp/src/arrow/ipc/CMakeLists.txt index 403ac6d4c0f..9ea0eebcb27 100644 --- a/cpp/src/arrow/ipc/CMakeLists.txt +++ b/cpp/src/arrow/ipc/CMakeLists.txt @@ -33,7 +33,6 @@ set(ARROW_IPC_SRCS json.cc json-internal.cc metadata.cc - metadata-internal.cc reader.cc writer.cc ) diff --git a/cpp/src/arrow/ipc/ipc-metadata-test.cc b/cpp/src/arrow/ipc/ipc-metadata-test.cc index 4fb3204a5b6..e98e25043cd 100644 --- a/cpp/src/arrow/ipc/ipc-metadata-test.cc +++ b/cpp/src/arrow/ipc/ipc-metadata-test.cc @@ -22,7 +22,6 @@ #include "gtest/gtest.h" #include "arrow/io/memory.h" -#include "arrow/ipc/metadata-internal.h" #include "arrow/ipc/metadata.h" #include "arrow/ipc/test-common.h" #include "arrow/schema.h" diff --git a/cpp/src/arrow/ipc/metadata-internal.cc b/cpp/src/arrow/ipc/metadata-internal.cc deleted file mode 100644 index be0d282f21b..00000000000 --- a/cpp/src/arrow/ipc/metadata-internal.cc +++ /dev/null @@ -1,597 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include "arrow/ipc/metadata-internal.h" - -#include -#include -#include -#include -#include - -#include "flatbuffers/flatbuffers.h" - -#include "arrow/array.h" -#include "arrow/buffer.h" -#include "arrow/ipc/Message_generated.h" -#include "arrow/schema.h" -#include "arrow/status.h" -#include "arrow/type.h" - -namespace arrow { - -namespace flatbuf = org::apache::arrow::flatbuf; - -namespace ipc { - -static Status IntFromFlatbuffer( - const flatbuf::Int* int_data, std::shared_ptr* out) { - if (int_data->bitWidth() > 64) { - return Status::NotImplemented("Integers with more than 64 bits not implemented"); - } - if (int_data->bitWidth() < 8) { - return Status::NotImplemented("Integers with less than 8 bits not implemented"); - } - - switch (int_data->bitWidth()) { - case 8: - *out = int_data->is_signed() ? int8() : uint8(); - break; - case 16: - *out = int_data->is_signed() ? int16() : uint16(); - break; - case 32: - *out = int_data->is_signed() ? int32() : uint32(); - break; - case 64: - *out = int_data->is_signed() ? int64() : uint64(); - break; - default: - return Status::NotImplemented("Integers not in cstdint are not implemented"); - } - return Status::OK(); -} - -static Status FloatFromFlatuffer( - const flatbuf::FloatingPoint* float_data, std::shared_ptr* out) { - if (float_data->precision() == flatbuf::Precision_HALF) { - *out = float16(); - } else if (float_data->precision() == flatbuf::Precision_SINGLE) { - *out = float32(); - } else { - *out = float64(); - } - return Status::OK(); -} - -// Forward declaration -static Status FieldToFlatbuffer(FBB& fbb, const std::shared_ptr& field, - DictionaryMemo* dictionary_memo, FieldOffset* offset); - -static Offset IntToFlatbuffer(FBB& fbb, int bitWidth, bool is_signed) { - return flatbuf::CreateInt(fbb, bitWidth, is_signed).Union(); -} - -static Offset FloatToFlatbuffer(FBB& fbb, flatbuf::Precision precision) { - return flatbuf::CreateFloatingPoint(fbb, precision).Union(); -} - -static Status AppendChildFields(FBB& fbb, const std::shared_ptr& type, - std::vector* out_children, DictionaryMemo* dictionary_memo) { - FieldOffset field; - for (int i = 0; i < type->num_children(); ++i) { - RETURN_NOT_OK(FieldToFlatbuffer(fbb, type->child(i), dictionary_memo, &field)); - out_children->push_back(field); - } - return Status::OK(); -} - -static Status ListToFlatbuffer(FBB& fbb, const std::shared_ptr& type, - std::vector* out_children, DictionaryMemo* dictionary_memo, - Offset* offset) { - RETURN_NOT_OK(AppendChildFields(fbb, type, out_children, dictionary_memo)); - *offset = flatbuf::CreateList(fbb).Union(); - return Status::OK(); -} - -static Status StructToFlatbuffer(FBB& fbb, const std::shared_ptr& type, - std::vector* out_children, DictionaryMemo* dictionary_memo, - Offset* offset) { - RETURN_NOT_OK(AppendChildFields(fbb, type, out_children, dictionary_memo)); - *offset = flatbuf::CreateStruct_(fbb).Union(); - return Status::OK(); -} - -// ---------------------------------------------------------------------- -// Union implementation - -static Status UnionFromFlatbuffer(const flatbuf::Union* union_data, - const std::vector>& children, std::shared_ptr* out) { - UnionMode mode = union_data->mode() == flatbuf::UnionMode_Sparse ? UnionMode::SPARSE - : UnionMode::DENSE; - - std::vector type_codes; - - const flatbuffers::Vector* fb_type_ids = union_data->typeIds(); - if (fb_type_ids == nullptr) { - for (uint8_t i = 0; i < children.size(); ++i) { - type_codes.push_back(i); - } - } else { - for (int32_t id : (*fb_type_ids)) { - // TODO(wesm): can these values exceed 255? - type_codes.push_back(static_cast(id)); - } - } - - *out = union_(children, type_codes, mode); - return Status::OK(); -} - -static Status UnionToFlatBuffer(FBB& fbb, const std::shared_ptr& type, - std::vector* out_children, DictionaryMemo* dictionary_memo, - Offset* offset) { - RETURN_NOT_OK(AppendChildFields(fbb, type, out_children, dictionary_memo)); - - const auto& union_type = static_cast(*type); - - flatbuf::UnionMode mode = union_type.mode == UnionMode::SPARSE - ? flatbuf::UnionMode_Sparse - : flatbuf::UnionMode_Dense; - - std::vector type_ids; - type_ids.reserve(union_type.type_codes.size()); - for (uint8_t code : union_type.type_codes) { - type_ids.push_back(code); - } - - auto fb_type_ids = fbb.CreateVector(type_ids); - - *offset = flatbuf::CreateUnion(fbb, mode, fb_type_ids).Union(); - return Status::OK(); -} - -#define INT_TO_FB_CASE(BIT_WIDTH, IS_SIGNED) \ - *out_type = flatbuf::Type_Int; \ - *offset = IntToFlatbuffer(fbb, BIT_WIDTH, IS_SIGNED); \ - break; - -static inline flatbuf::TimeUnit ToFlatbufferUnit(TimeUnit unit) { - switch (unit) { - case TimeUnit::SECOND: - return flatbuf::TimeUnit_SECOND; - case TimeUnit::MILLI: - return flatbuf::TimeUnit_MILLISECOND; - case TimeUnit::MICRO: - return flatbuf::TimeUnit_MICROSECOND; - case TimeUnit::NANO: - return flatbuf::TimeUnit_NANOSECOND; - default: - break; - } - return flatbuf::TimeUnit_MIN; -} - -static inline TimeUnit FromFlatbufferUnit(flatbuf::TimeUnit unit) { - switch (unit) { - case flatbuf::TimeUnit_SECOND: - return TimeUnit::SECOND; - case flatbuf::TimeUnit_MILLISECOND: - return TimeUnit::MILLI; - case flatbuf::TimeUnit_MICROSECOND: - return TimeUnit::MICRO; - case flatbuf::TimeUnit_NANOSECOND: - return TimeUnit::NANO; - default: - break; - } - // cannot reach - return TimeUnit::SECOND; -} - -static Status TypeFromFlatbuffer(flatbuf::Type type, const void* type_data, - const std::vector>& children, std::shared_ptr* out) { - switch (type) { - case flatbuf::Type_NONE: - return Status::Invalid("Type metadata cannot be none"); - case flatbuf::Type_Int: - return IntFromFlatbuffer(static_cast(type_data), out); - case flatbuf::Type_FloatingPoint: - return FloatFromFlatuffer( - static_cast(type_data), out); - case flatbuf::Type_Binary: - *out = binary(); - return Status::OK(); - case flatbuf::Type_FixedWidthBinary: { - auto fw_binary = static_cast(type_data); - *out = fixed_width_binary(fw_binary->byteWidth()); - return Status::OK(); - } - case flatbuf::Type_Utf8: - *out = utf8(); - return Status::OK(); - case flatbuf::Type_Bool: - *out = boolean(); - return Status::OK(); - case flatbuf::Type_Decimal: - return Status::NotImplemented("Decimal"); - case flatbuf::Type_Date: - *out = date(); - return Status::OK(); - case flatbuf::Type_Time: { - auto time_type = static_cast(type_data); - *out = time(FromFlatbufferUnit(time_type->unit())); - return Status::OK(); - } - case flatbuf::Type_Timestamp: { - auto ts_type = static_cast(type_data); - *out = timestamp(FromFlatbufferUnit(ts_type->unit())); - return Status::OK(); - } - case flatbuf::Type_Interval: - return Status::NotImplemented("Interval"); - case flatbuf::Type_List: - if (children.size() != 1) { - return Status::Invalid("List must have exactly 1 child field"); - } - *out = std::make_shared(children[0]); - return Status::OK(); - case flatbuf::Type_Struct_: - *out = std::make_shared(children); - return Status::OK(); - case flatbuf::Type_Union: - return UnionFromFlatbuffer( - static_cast(type_data), children, out); - default: - return Status::Invalid("Unrecognized type"); - } -} - -// TODO(wesm): Convert this to visitor pattern -static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr& type, - std::vector* children, std::vector* layout, - flatbuf::Type* out_type, DictionaryMemo* dictionary_memo, Offset* offset) { - if (type->type == Type::DICTIONARY) { - // In this library, the dictionary "type" is a logical construct. Here we - // pass through to the value type, as we've already captured the index - // type in the DictionaryEncoding metadata in the parent field - const auto& dict_type = static_cast(*type); - return TypeToFlatbuffer(fbb, dict_type.dictionary()->type(), children, layout, - out_type, dictionary_memo, offset); - } - - std::vector buffer_layout = type->GetBufferLayout(); - for (const BufferDescr& descr : buffer_layout) { - flatbuf::VectorType vector_type; - switch (descr.type()) { - case BufferType::OFFSET: - vector_type = flatbuf::VectorType_OFFSET; - break; - case BufferType::DATA: - vector_type = flatbuf::VectorType_DATA; - break; - case BufferType::VALIDITY: - vector_type = flatbuf::VectorType_VALIDITY; - break; - case BufferType::TYPE: - vector_type = flatbuf::VectorType_TYPE; - break; - default: - vector_type = flatbuf::VectorType_DATA; - break; - } - auto offset = flatbuf::CreateVectorLayout( - fbb, static_cast(descr.bit_width()), vector_type); - layout->push_back(offset); - } - - switch (type->type) { - case Type::BOOL: - *out_type = flatbuf::Type_Bool; - *offset = flatbuf::CreateBool(fbb).Union(); - break; - case Type::UINT8: - INT_TO_FB_CASE(8, false); - case Type::INT8: - INT_TO_FB_CASE(8, true); - case Type::UINT16: - INT_TO_FB_CASE(16, false); - case Type::INT16: - INT_TO_FB_CASE(16, true); - case Type::UINT32: - INT_TO_FB_CASE(32, false); - case Type::INT32: - INT_TO_FB_CASE(32, true); - case Type::UINT64: - INT_TO_FB_CASE(64, false); - case Type::INT64: - INT_TO_FB_CASE(64, true); - case Type::FLOAT: - *out_type = flatbuf::Type_FloatingPoint; - *offset = FloatToFlatbuffer(fbb, flatbuf::Precision_SINGLE); - break; - case Type::DOUBLE: - *out_type = flatbuf::Type_FloatingPoint; - *offset = FloatToFlatbuffer(fbb, flatbuf::Precision_DOUBLE); - break; - case Type::FIXED_WIDTH_BINARY: { - const auto& fw_type = static_cast(*type); - *out_type = flatbuf::Type_FixedWidthBinary; - *offset = flatbuf::CreateFixedWidthBinary(fbb, fw_type.byte_width()).Union(); - } break; - case Type::BINARY: - *out_type = flatbuf::Type_Binary; - *offset = flatbuf::CreateBinary(fbb).Union(); - break; - case Type::STRING: - *out_type = flatbuf::Type_Utf8; - *offset = flatbuf::CreateUtf8(fbb).Union(); - break; - case Type::DATE: - *out_type = flatbuf::Type_Date; - *offset = flatbuf::CreateDate(fbb).Union(); - break; - case Type::TIME: { - const auto& time_type = static_cast(*type); - *out_type = flatbuf::Type_Time; - *offset = flatbuf::CreateTime(fbb, ToFlatbufferUnit(time_type.unit)).Union(); - } break; - case Type::TIMESTAMP: { - const auto& ts_type = static_cast(*type); - *out_type = flatbuf::Type_Timestamp; - *offset = flatbuf::CreateTimestamp(fbb, ToFlatbufferUnit(ts_type.unit)).Union(); - } break; - case Type::LIST: - *out_type = flatbuf::Type_List; - return ListToFlatbuffer(fbb, type, children, dictionary_memo, offset); - case Type::STRUCT: - *out_type = flatbuf::Type_Struct_; - return StructToFlatbuffer(fbb, type, children, dictionary_memo, offset); - case Type::UNION: - *out_type = flatbuf::Type_Union; - return UnionToFlatBuffer(fbb, type, children, dictionary_memo, offset); - default: - *out_type = flatbuf::Type_NONE; // Make clang-tidy happy - std::stringstream ss; - ss << "Unable to convert type: " << type->ToString() << std::endl; - return Status::NotImplemented(ss.str()); - } - return Status::OK(); -} - -using DictionaryOffset = flatbuffers::Offset; - -static DictionaryOffset GetDictionaryEncoding( - FBB& fbb, const DictionaryType& type, DictionaryMemo* memo) { - int64_t dictionary_id = memo->GetId(type.dictionary()); - - // We assume that the dictionary index type (as an integer) has already been - // validated elsewhere, and can safely assume we are dealing with signed - // integers - const auto& fw_index_type = static_cast(*type.index_type()); - - auto index_type_offset = flatbuf::CreateInt(fbb, fw_index_type.bit_width(), true); - - // TODO(wesm): ordered dictionaries - return flatbuf::CreateDictionaryEncoding(fbb, dictionary_id, index_type_offset); -} - -static Status FieldToFlatbuffer(FBB& fbb, const std::shared_ptr& field, - DictionaryMemo* dictionary_memo, FieldOffset* offset) { - auto fb_name = fbb.CreateString(field->name); - - flatbuf::Type type_enum; - Offset type_offset; - Offset type_layout; - std::vector children; - std::vector layout; - - RETURN_NOT_OK(TypeToFlatbuffer( - fbb, field->type, &children, &layout, &type_enum, dictionary_memo, &type_offset)); - auto fb_children = fbb.CreateVector(children); - auto fb_layout = fbb.CreateVector(layout); - - DictionaryOffset dictionary = 0; - if (field->type->type == Type::DICTIONARY) { - dictionary = GetDictionaryEncoding( - fbb, static_cast(*field->type), dictionary_memo); - } - - // TODO: produce the list of VectorTypes - *offset = flatbuf::CreateField(fbb, fb_name, field->nullable, type_enum, type_offset, - dictionary, fb_children, fb_layout); - - return Status::OK(); -} - -Status FieldFromFlatbufferDictionary( - const flatbuf::Field* field, std::shared_ptr* out) { - // Need an empty memo to pass down for constructing children - DictionaryMemo dummy_memo; - - // Any DictionaryEncoding set is ignored here - - std::shared_ptr type; - auto children = field->children(); - std::vector> child_fields(children->size()); - for (int i = 0; i < static_cast(children->size()); ++i) { - RETURN_NOT_OK(FieldFromFlatbuffer(children->Get(i), dummy_memo, &child_fields[i])); - } - - RETURN_NOT_OK( - TypeFromFlatbuffer(field->type_type(), field->type(), child_fields, &type)); - - *out = std::make_shared(field->name()->str(), type, field->nullable()); - return Status::OK(); -} - -Status FieldFromFlatbuffer(const flatbuf::Field* field, - const DictionaryMemo& dictionary_memo, std::shared_ptr* out) { - std::shared_ptr type; - - const flatbuf::DictionaryEncoding* encoding = field->dictionary(); - - if (encoding == nullptr) { - // The field is not dictionary encoded. We must potentially visit its - // children to fully reconstruct the data type - auto children = field->children(); - std::vector> child_fields(children->size()); - for (int i = 0; i < static_cast(children->size()); ++i) { - RETURN_NOT_OK( - FieldFromFlatbuffer(children->Get(i), dictionary_memo, &child_fields[i])); - } - RETURN_NOT_OK( - TypeFromFlatbuffer(field->type_type(), field->type(), child_fields, &type)); - } else { - // The field is dictionary encoded. The type of the dictionary values has - // been determined elsewhere, and is stored in the DictionaryMemo. Here we - // construct the logical DictionaryType object - - std::shared_ptr dictionary; - RETURN_NOT_OK(dictionary_memo.GetDictionary(encoding->id(), &dictionary)); - - std::shared_ptr index_type; - RETURN_NOT_OK(IntFromFlatbuffer(encoding->indexType(), &index_type)); - type = std::make_shared(index_type, dictionary); - } - *out = std::make_shared(field->name()->str(), type, field->nullable()); - return Status::OK(); -} - -// Implement MessageBuilder - -// will return the endianness of the system we are running on -// based the NUMPY_API function. See NOTICE.txt -flatbuf::Endianness endianness() { - union { - uint32_t i; - char c[4]; - } bint = {0x01020304}; - - return bint.c[0] == 1 ? flatbuf::Endianness_Big : flatbuf::Endianness_Little; -} - -Status SchemaToFlatbuffer(FBB& fbb, const Schema& schema, DictionaryMemo* dictionary_memo, - flatbuffers::Offset* out) { - std::vector field_offsets; - for (int i = 0; i < schema.num_fields(); ++i) { - std::shared_ptr field = schema.field(i); - FieldOffset offset; - RETURN_NOT_OK(FieldToFlatbuffer(fbb, field, dictionary_memo, &offset)); - field_offsets.push_back(offset); - } - - *out = flatbuf::CreateSchema(fbb, endianness(), fbb.CreateVector(field_offsets)); - return Status::OK(); -} - -class MessageBuilder { - public: - Status SetSchema(const Schema& schema, DictionaryMemo* dictionary_memo) { - flatbuffers::Offset fb_schema; - RETURN_NOT_OK(SchemaToFlatbuffer(fbb_, schema, dictionary_memo, &fb_schema)); - - header_type_ = flatbuf::MessageHeader_Schema; - header_ = fb_schema.Union(); - body_length_ = 0; - return Status::OK(); - } - - Status SetRecordBatch(int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers) { - header_type_ = flatbuf::MessageHeader_RecordBatch; - header_ = flatbuf::CreateRecordBatch(fbb_, length, fbb_.CreateVectorOfStructs(nodes), - fbb_.CreateVectorOfStructs(buffers)) - .Union(); - body_length_ = body_length; - - return Status::OK(); - } - - Status SetDictionary(int64_t id, int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers) { - header_type_ = flatbuf::MessageHeader_DictionaryBatch; - - auto record_batch = flatbuf::CreateRecordBatch(fbb_, length, - fbb_.CreateVectorOfStructs(nodes), fbb_.CreateVectorOfStructs(buffers)); - - header_ = flatbuf::CreateDictionaryBatch(fbb_, id, record_batch).Union(); - body_length_ = body_length; - return Status::OK(); - } - - Status Finish(); - - Status GetBuffer(std::shared_ptr* out); - - private: - flatbuf::MessageHeader header_type_; - flatbuffers::Offset header_; - int64_t body_length_; - flatbuffers::FlatBufferBuilder fbb_; -}; - -Status WriteSchemaMessage( - const Schema& schema, DictionaryMemo* dictionary_memo, std::shared_ptr* out) { - MessageBuilder message; - RETURN_NOT_OK(message.SetSchema(schema, dictionary_memo)); - RETURN_NOT_OK(message.Finish()); - return message.GetBuffer(out); -} - -Status WriteRecordBatchMessage(int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers, std::shared_ptr* out) { - MessageBuilder builder; - RETURN_NOT_OK(builder.SetRecordBatch(length, body_length, nodes, buffers)); - RETURN_NOT_OK(builder.Finish()); - return builder.GetBuffer(out); -} - -Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers, std::shared_ptr* out) { - MessageBuilder builder; - RETURN_NOT_OK(builder.SetDictionary(id, length, body_length, nodes, buffers)); - RETURN_NOT_OK(builder.Finish()); - return builder.GetBuffer(out); -} - -Status MessageBuilder::Finish() { - auto message = - flatbuf::CreateMessage(fbb_, kMetadataVersion, header_type_, header_, body_length_); - fbb_.Finish(message); - return Status::OK(); -} - -Status MessageBuilder::GetBuffer(std::shared_ptr* out) { - int32_t size = fbb_.GetSize(); - - auto result = std::make_shared(); - RETURN_NOT_OK(result->Resize(size)); - - uint8_t* dst = result->mutable_data(); - memcpy(dst, fbb_.GetBufferPointer(), size); - - *out = result; - return Status::OK(); -} - -} // namespace ipc -} // namespace arrow diff --git a/cpp/src/arrow/ipc/metadata-internal.h b/cpp/src/arrow/ipc/metadata-internal.h deleted file mode 100644 index 59afecbcbd2..00000000000 --- a/cpp/src/arrow/ipc/metadata-internal.h +++ /dev/null @@ -1,83 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#ifndef ARROW_IPC_METADATA_INTERNAL_H -#define ARROW_IPC_METADATA_INTERNAL_H - -#include -#include -#include - -#include "flatbuffers/flatbuffers.h" - -#include "arrow/ipc/File_generated.h" -#include "arrow/ipc/Message_generated.h" -#include "arrow/ipc/metadata.h" - -namespace arrow { - -namespace flatbuf = org::apache::arrow::flatbuf; - -class Buffer; -struct Field; -class Schema; -class Status; - -namespace ipc { - -using FBB = flatbuffers::FlatBufferBuilder; -using FieldOffset = flatbuffers::Offset; -using VectorLayoutOffset = flatbuffers::Offset; -using Offset = flatbuffers::Offset; - -static constexpr flatbuf::MetadataVersion kMetadataVersion = flatbuf::MetadataVersion_V2; - -// Construct a field with type for a dictionary-encoded field. None of its -// children or children's descendents can be dictionary encoded -Status FieldFromFlatbufferDictionary( - const flatbuf::Field* field, std::shared_ptr* out); - -// Construct a field for a non-dictionary-encoded field. Its children may be -// dictionary encoded -Status FieldFromFlatbuffer(const flatbuf::Field* field, - const DictionaryMemo& dictionary_memo, std::shared_ptr* out); - -Status SchemaToFlatbuffer(FBB& fbb, const Schema& schema, DictionaryMemo* dictionary_memo, - flatbuffers::Offset* out); - -// Serialize arrow::Schema as a Flatbuffer -// -// \param[in] schema a Schema instance -// \param[inout] dictionary_memo class for tracking dictionaries and assigning -// dictionary ids -// \param[out] out the serialized arrow::Buffer -// \return Status outcome -Status WriteSchemaMessage( - const Schema& schema, DictionaryMemo* dictionary_memo, std::shared_ptr* out); - -Status WriteRecordBatchMessage(int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers, std::shared_ptr* out); - -Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers, std::shared_ptr* out); - -} // namespace ipc -} // namespace arrow - -#endif // ARROW_IPC_METADATA_INTERNAL_H diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc index 2b42d18713b..a1f54f07ba4 100644 --- a/cpp/src/arrow/ipc/metadata.cc +++ b/cpp/src/arrow/ipc/metadata.cc @@ -24,14 +24,14 @@ #include "flatbuffers/flatbuffers.h" +#include "arrow/array.h" +#include "arrow/buffer.h" #include "arrow/io/interfaces.h" #include "arrow/ipc/File_generated.h" #include "arrow/ipc/Message_generated.h" -#include "arrow/ipc/metadata-internal.h" - -#include "arrow/buffer.h" #include "arrow/schema.h" #include "arrow/status.h" +#include "arrow/type.h" namespace arrow { @@ -39,6 +39,625 @@ namespace flatbuf = org::apache::arrow::flatbuf; namespace ipc { +using FBB = flatbuffers::FlatBufferBuilder; +using DictionaryOffset = flatbuffers::Offset; +using FieldOffset = flatbuffers::Offset; +using RecordBatchOffset = flatbuffers::Offset; +using VectorLayoutOffset = flatbuffers::Offset; +using Offset = flatbuffers::Offset; + +static constexpr flatbuf::MetadataVersion kMetadataVersion = flatbuf::MetadataVersion_V2; + +static Status IntFromFlatbuffer( + const flatbuf::Int* int_data, std::shared_ptr* out) { + if (int_data->bitWidth() > 64) { + return Status::NotImplemented("Integers with more than 64 bits not implemented"); + } + if (int_data->bitWidth() < 8) { + return Status::NotImplemented("Integers with less than 8 bits not implemented"); + } + + switch (int_data->bitWidth()) { + case 8: + *out = int_data->is_signed() ? int8() : uint8(); + break; + case 16: + *out = int_data->is_signed() ? int16() : uint16(); + break; + case 32: + *out = int_data->is_signed() ? int32() : uint32(); + break; + case 64: + *out = int_data->is_signed() ? int64() : uint64(); + break; + default: + return Status::NotImplemented("Integers not in cstdint are not implemented"); + } + return Status::OK(); +} + +static Status FloatFromFlatuffer( + const flatbuf::FloatingPoint* float_data, std::shared_ptr* out) { + if (float_data->precision() == flatbuf::Precision_HALF) { + *out = float16(); + } else if (float_data->precision() == flatbuf::Precision_SINGLE) { + *out = float32(); + } else { + *out = float64(); + } + return Status::OK(); +} + +// Forward declaration +static Status FieldToFlatbuffer(FBB& fbb, const std::shared_ptr& field, + DictionaryMemo* dictionary_memo, FieldOffset* offset); + +static Offset IntToFlatbuffer(FBB& fbb, int bitWidth, bool is_signed) { + return flatbuf::CreateInt(fbb, bitWidth, is_signed).Union(); +} + +static Offset FloatToFlatbuffer(FBB& fbb, flatbuf::Precision precision) { + return flatbuf::CreateFloatingPoint(fbb, precision).Union(); +} + +static Status AppendChildFields(FBB& fbb, const std::shared_ptr& type, + std::vector* out_children, DictionaryMemo* dictionary_memo) { + FieldOffset field; + for (int i = 0; i < type->num_children(); ++i) { + RETURN_NOT_OK(FieldToFlatbuffer(fbb, type->child(i), dictionary_memo, &field)); + out_children->push_back(field); + } + return Status::OK(); +} + +static Status ListToFlatbuffer(FBB& fbb, const std::shared_ptr& type, + std::vector* out_children, DictionaryMemo* dictionary_memo, + Offset* offset) { + RETURN_NOT_OK(AppendChildFields(fbb, type, out_children, dictionary_memo)); + *offset = flatbuf::CreateList(fbb).Union(); + return Status::OK(); +} + +static Status StructToFlatbuffer(FBB& fbb, const std::shared_ptr& type, + std::vector* out_children, DictionaryMemo* dictionary_memo, + Offset* offset) { + RETURN_NOT_OK(AppendChildFields(fbb, type, out_children, dictionary_memo)); + *offset = flatbuf::CreateStruct_(fbb).Union(); + return Status::OK(); +} + +// ---------------------------------------------------------------------- +// Union implementation + +static Status UnionFromFlatbuffer(const flatbuf::Union* union_data, + const std::vector>& children, std::shared_ptr* out) { + UnionMode mode = union_data->mode() == flatbuf::UnionMode_Sparse ? UnionMode::SPARSE + : UnionMode::DENSE; + + std::vector type_codes; + + const flatbuffers::Vector* fb_type_ids = union_data->typeIds(); + if (fb_type_ids == nullptr) { + for (uint8_t i = 0; i < children.size(); ++i) { + type_codes.push_back(i); + } + } else { + for (int32_t id : (*fb_type_ids)) { + // TODO(wesm): can these values exceed 255? + type_codes.push_back(static_cast(id)); + } + } + + *out = union_(children, type_codes, mode); + return Status::OK(); +} + +static Status UnionToFlatBuffer(FBB& fbb, const std::shared_ptr& type, + std::vector* out_children, DictionaryMemo* dictionary_memo, + Offset* offset) { + RETURN_NOT_OK(AppendChildFields(fbb, type, out_children, dictionary_memo)); + + const auto& union_type = static_cast(*type); + + flatbuf::UnionMode mode = union_type.mode == UnionMode::SPARSE + ? flatbuf::UnionMode_Sparse + : flatbuf::UnionMode_Dense; + + std::vector type_ids; + type_ids.reserve(union_type.type_codes.size()); + for (uint8_t code : union_type.type_codes) { + type_ids.push_back(code); + } + + auto fb_type_ids = fbb.CreateVector(type_ids); + + *offset = flatbuf::CreateUnion(fbb, mode, fb_type_ids).Union(); + return Status::OK(); +} + +#define INT_TO_FB_CASE(BIT_WIDTH, IS_SIGNED) \ + *out_type = flatbuf::Type_Int; \ + *offset = IntToFlatbuffer(fbb, BIT_WIDTH, IS_SIGNED); \ + break; + +static inline flatbuf::TimeUnit ToFlatbufferUnit(TimeUnit unit) { + switch (unit) { + case TimeUnit::SECOND: + return flatbuf::TimeUnit_SECOND; + case TimeUnit::MILLI: + return flatbuf::TimeUnit_MILLISECOND; + case TimeUnit::MICRO: + return flatbuf::TimeUnit_MICROSECOND; + case TimeUnit::NANO: + return flatbuf::TimeUnit_NANOSECOND; + default: + break; + } + return flatbuf::TimeUnit_MIN; +} + +static inline TimeUnit FromFlatbufferUnit(flatbuf::TimeUnit unit) { + switch (unit) { + case flatbuf::TimeUnit_SECOND: + return TimeUnit::SECOND; + case flatbuf::TimeUnit_MILLISECOND: + return TimeUnit::MILLI; + case flatbuf::TimeUnit_MICROSECOND: + return TimeUnit::MICRO; + case flatbuf::TimeUnit_NANOSECOND: + return TimeUnit::NANO; + default: + break; + } + // cannot reach + return TimeUnit::SECOND; +} + +static Status TypeFromFlatbuffer(flatbuf::Type type, const void* type_data, + const std::vector>& children, std::shared_ptr* out) { + switch (type) { + case flatbuf::Type_NONE: + return Status::Invalid("Type metadata cannot be none"); + case flatbuf::Type_Int: + return IntFromFlatbuffer(static_cast(type_data), out); + case flatbuf::Type_FloatingPoint: + return FloatFromFlatuffer( + static_cast(type_data), out); + case flatbuf::Type_Binary: + *out = binary(); + return Status::OK(); + case flatbuf::Type_FixedWidthBinary: { + auto fw_binary = static_cast(type_data); + *out = fixed_width_binary(fw_binary->byteWidth()); + return Status::OK(); + } + case flatbuf::Type_Utf8: + *out = utf8(); + return Status::OK(); + case flatbuf::Type_Bool: + *out = boolean(); + return Status::OK(); + case flatbuf::Type_Decimal: + return Status::NotImplemented("Decimal"); + case flatbuf::Type_Date: + *out = date(); + return Status::OK(); + case flatbuf::Type_Time: { + auto time_type = static_cast(type_data); + *out = time(FromFlatbufferUnit(time_type->unit())); + return Status::OK(); + } + case flatbuf::Type_Timestamp: { + auto ts_type = static_cast(type_data); + *out = timestamp(FromFlatbufferUnit(ts_type->unit())); + return Status::OK(); + } + case flatbuf::Type_Interval: + return Status::NotImplemented("Interval"); + case flatbuf::Type_List: + if (children.size() != 1) { + return Status::Invalid("List must have exactly 1 child field"); + } + *out = std::make_shared(children[0]); + return Status::OK(); + case flatbuf::Type_Struct_: + *out = std::make_shared(children); + return Status::OK(); + case flatbuf::Type_Union: + return UnionFromFlatbuffer( + static_cast(type_data), children, out); + default: + return Status::Invalid("Unrecognized type"); + } +} + +// TODO(wesm): Convert this to visitor pattern +static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr& type, + std::vector* children, std::vector* layout, + flatbuf::Type* out_type, DictionaryMemo* dictionary_memo, Offset* offset) { + if (type->type == Type::DICTIONARY) { + // In this library, the dictionary "type" is a logical construct. Here we + // pass through to the value type, as we've already captured the index + // type in the DictionaryEncoding metadata in the parent field + const auto& dict_type = static_cast(*type); + return TypeToFlatbuffer(fbb, dict_type.dictionary()->type(), children, layout, + out_type, dictionary_memo, offset); + } + + std::vector buffer_layout = type->GetBufferLayout(); + for (const BufferDescr& descr : buffer_layout) { + flatbuf::VectorType vector_type; + switch (descr.type()) { + case BufferType::OFFSET: + vector_type = flatbuf::VectorType_OFFSET; + break; + case BufferType::DATA: + vector_type = flatbuf::VectorType_DATA; + break; + case BufferType::VALIDITY: + vector_type = flatbuf::VectorType_VALIDITY; + break; + case BufferType::TYPE: + vector_type = flatbuf::VectorType_TYPE; + break; + default: + vector_type = flatbuf::VectorType_DATA; + break; + } + auto offset = flatbuf::CreateVectorLayout( + fbb, static_cast(descr.bit_width()), vector_type); + layout->push_back(offset); + } + + switch (type->type) { + case Type::BOOL: + *out_type = flatbuf::Type_Bool; + *offset = flatbuf::CreateBool(fbb).Union(); + break; + case Type::UINT8: + INT_TO_FB_CASE(8, false); + case Type::INT8: + INT_TO_FB_CASE(8, true); + case Type::UINT16: + INT_TO_FB_CASE(16, false); + case Type::INT16: + INT_TO_FB_CASE(16, true); + case Type::UINT32: + INT_TO_FB_CASE(32, false); + case Type::INT32: + INT_TO_FB_CASE(32, true); + case Type::UINT64: + INT_TO_FB_CASE(64, false); + case Type::INT64: + INT_TO_FB_CASE(64, true); + case Type::FLOAT: + *out_type = flatbuf::Type_FloatingPoint; + *offset = FloatToFlatbuffer(fbb, flatbuf::Precision_SINGLE); + break; + case Type::DOUBLE: + *out_type = flatbuf::Type_FloatingPoint; + *offset = FloatToFlatbuffer(fbb, flatbuf::Precision_DOUBLE); + break; + case Type::FIXED_WIDTH_BINARY: { + const auto& fw_type = static_cast(*type); + *out_type = flatbuf::Type_FixedWidthBinary; + *offset = flatbuf::CreateFixedWidthBinary(fbb, fw_type.byte_width()).Union(); + } break; + case Type::BINARY: + *out_type = flatbuf::Type_Binary; + *offset = flatbuf::CreateBinary(fbb).Union(); + break; + case Type::STRING: + *out_type = flatbuf::Type_Utf8; + *offset = flatbuf::CreateUtf8(fbb).Union(); + break; + case Type::DATE: + *out_type = flatbuf::Type_Date; + *offset = flatbuf::CreateDate(fbb).Union(); + break; + case Type::TIME: { + const auto& time_type = static_cast(*type); + *out_type = flatbuf::Type_Time; + *offset = flatbuf::CreateTime(fbb, ToFlatbufferUnit(time_type.unit)).Union(); + } break; + case Type::TIMESTAMP: { + const auto& ts_type = static_cast(*type); + *out_type = flatbuf::Type_Timestamp; + *offset = flatbuf::CreateTimestamp(fbb, ToFlatbufferUnit(ts_type.unit)).Union(); + } break; + case Type::LIST: + *out_type = flatbuf::Type_List; + return ListToFlatbuffer(fbb, type, children, dictionary_memo, offset); + case Type::STRUCT: + *out_type = flatbuf::Type_Struct_; + return StructToFlatbuffer(fbb, type, children, dictionary_memo, offset); + case Type::UNION: + *out_type = flatbuf::Type_Union; + return UnionToFlatBuffer(fbb, type, children, dictionary_memo, offset); + default: + *out_type = flatbuf::Type_NONE; // Make clang-tidy happy + std::stringstream ss; + ss << "Unable to convert type: " << type->ToString() << std::endl; + return Status::NotImplemented(ss.str()); + } + return Status::OK(); +} + +static DictionaryOffset GetDictionaryEncoding( + FBB& fbb, const DictionaryType& type, DictionaryMemo* memo) { + int64_t dictionary_id = memo->GetId(type.dictionary()); + + // We assume that the dictionary index type (as an integer) has already been + // validated elsewhere, and can safely assume we are dealing with signed + // integers + const auto& fw_index_type = static_cast(*type.index_type()); + + auto index_type_offset = flatbuf::CreateInt(fbb, fw_index_type.bit_width(), true); + + // TODO(wesm): ordered dictionaries + return flatbuf::CreateDictionaryEncoding(fbb, dictionary_id, index_type_offset); +} + +static Status FieldToFlatbuffer(FBB& fbb, const std::shared_ptr& field, + DictionaryMemo* dictionary_memo, FieldOffset* offset) { + auto fb_name = fbb.CreateString(field->name); + + flatbuf::Type type_enum; + Offset type_offset; + Offset type_layout; + std::vector children; + std::vector layout; + + RETURN_NOT_OK(TypeToFlatbuffer( + fbb, field->type, &children, &layout, &type_enum, dictionary_memo, &type_offset)); + auto fb_children = fbb.CreateVector(children); + auto fb_layout = fbb.CreateVector(layout); + + DictionaryOffset dictionary = 0; + if (field->type->type == Type::DICTIONARY) { + dictionary = GetDictionaryEncoding( + fbb, static_cast(*field->type), dictionary_memo); + } + + // TODO: produce the list of VectorTypes + *offset = flatbuf::CreateField(fbb, fb_name, field->nullable, type_enum, type_offset, + dictionary, fb_children, fb_layout); + + return Status::OK(); +} + +static Status FieldFromFlatbuffer(const flatbuf::Field* field, + const DictionaryMemo& dictionary_memo, std::shared_ptr* out) { + std::shared_ptr type; + + const flatbuf::DictionaryEncoding* encoding = field->dictionary(); + + if (encoding == nullptr) { + // The field is not dictionary encoded. We must potentially visit its + // children to fully reconstruct the data type + auto children = field->children(); + std::vector> child_fields(children->size()); + for (int i = 0; i < static_cast(children->size()); ++i) { + RETURN_NOT_OK( + FieldFromFlatbuffer(children->Get(i), dictionary_memo, &child_fields[i])); + } + RETURN_NOT_OK( + TypeFromFlatbuffer(field->type_type(), field->type(), child_fields, &type)); + } else { + // The field is dictionary encoded. The type of the dictionary values has + // been determined elsewhere, and is stored in the DictionaryMemo. Here we + // construct the logical DictionaryType object + + std::shared_ptr dictionary; + RETURN_NOT_OK(dictionary_memo.GetDictionary(encoding->id(), &dictionary)); + + std::shared_ptr index_type; + RETURN_NOT_OK(IntFromFlatbuffer(encoding->indexType(), &index_type)); + type = std::make_shared(index_type, dictionary); + } + *out = std::make_shared(field->name()->str(), type, field->nullable()); + return Status::OK(); +} + +static Status FieldFromFlatbufferDictionary( + const flatbuf::Field* field, std::shared_ptr* out) { + // Need an empty memo to pass down for constructing children + DictionaryMemo dummy_memo; + + // Any DictionaryEncoding set is ignored here + + std::shared_ptr type; + auto children = field->children(); + std::vector> child_fields(children->size()); + for (int i = 0; i < static_cast(children->size()); ++i) { + RETURN_NOT_OK(FieldFromFlatbuffer(children->Get(i), dummy_memo, &child_fields[i])); + } + + RETURN_NOT_OK( + TypeFromFlatbuffer(field->type_type(), field->type(), child_fields, &type)); + + *out = std::make_shared(field->name()->str(), type, field->nullable()); + return Status::OK(); +} + +// Implement MessageBuilder + +// will return the endianness of the system we are running on +// based the NUMPY_API function. See NOTICE.txt +flatbuf::Endianness endianness() { + union { + uint32_t i; + char c[4]; + } bint = {0x01020304}; + + return bint.c[0] == 1 ? flatbuf::Endianness_Big : flatbuf::Endianness_Little; +} + +static Status SchemaToFlatbuffer(FBB& fbb, const Schema& schema, + DictionaryMemo* dictionary_memo, flatbuffers::Offset* out) { + std::vector field_offsets; + for (int i = 0; i < schema.num_fields(); ++i) { + std::shared_ptr field = schema.field(i); + FieldOffset offset; + RETURN_NOT_OK(FieldToFlatbuffer(fbb, field, dictionary_memo, &offset)); + field_offsets.push_back(offset); + } + + *out = flatbuf::CreateSchema(fbb, endianness(), fbb.CreateVector(field_offsets)); + return Status::OK(); +} + +class MessageBuilder { + public: + Status SetSchema(const Schema& schema, DictionaryMemo* dictionary_memo) { + flatbuffers::Offset fb_schema; + RETURN_NOT_OK(SchemaToFlatbuffer(fbb_, schema, dictionary_memo, &fb_schema)); + + header_type_ = flatbuf::MessageHeader_Schema; + header_ = fb_schema.Union(); + body_length_ = 0; + return Status::OK(); + } + + Status MakeRecordBatch(int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + RecordBatchOffset* offset) { + std::vector fb_nodes; + std::vector fb_buffers; + fb_nodes.reserve(nodes.size()); + fb_buffers.reserve(buffers.size()); + + for (size_t i = 0; i < nodes.size(); ++i) { + const FieldMetadata& node = nodes[i]; + if (node.offset != 0) { + return Status::Invalid("Field metadata for IPC must have offset 0"); + } + fb_nodes.emplace_back(node.length, node.null_count); + } + + for (size_t i = 0; i < buffers.size(); ++i) { + const BufferMetadata& buffer = buffers[i]; + fb_buffers.emplace_back(buffer.page, buffer.offset, buffer.length); + } + *offset = flatbuf::CreateRecordBatch(fbb_, length, + fbb_.CreateVectorOfStructs(fb_nodes), fbb_.CreateVectorOfStructs(fb_buffers)); + return Status::OK(); + } + + Status SetRecordBatch(int32_t length, int64_t body_length, + const std::vector& nodes, + const std::vector& buffers) { + header_type_ = flatbuf::MessageHeader_RecordBatch; + + RecordBatchOffset batch_offset; + RETURN_NOT_OK(MakeRecordBatch(length, body_length, nodes, buffers, &batch_offset)); + header_ = batch_offset.Union(); + body_length_ = body_length; + + return Status::OK(); + } + + Status SetDictionary(int64_t id, int32_t length, int64_t body_length, + const std::vector& nodes, + const std::vector& buffers) { + header_type_ = flatbuf::MessageHeader_DictionaryBatch; + + RecordBatchOffset record_batch; + RETURN_NOT_OK(MakeRecordBatch(length, body_length, nodes, buffers, &record_batch)); + header_ = flatbuf::CreateDictionaryBatch(fbb_, id, record_batch).Union(); + body_length_ = body_length; + return Status::OK(); + } + + Status Finish(); + + Status GetBuffer(std::shared_ptr* out); + + private: + flatbuf::MessageHeader header_type_; + flatbuffers::Offset header_; + int64_t body_length_; + flatbuffers::FlatBufferBuilder fbb_; +}; + +Status WriteSchemaMessage( + const Schema& schema, DictionaryMemo* dictionary_memo, std::shared_ptr* out) { + MessageBuilder message; + RETURN_NOT_OK(message.SetSchema(schema, dictionary_memo)); + RETURN_NOT_OK(message.Finish()); + return message.GetBuffer(out); +} + +Status WriteRecordBatchMessage(int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + std::shared_ptr* out) { + MessageBuilder builder; + RETURN_NOT_OK(builder.SetRecordBatch(length, body_length, nodes, buffers)); + RETURN_NOT_OK(builder.Finish()); + return builder.GetBuffer(out); +} + +Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + std::shared_ptr* out) { + MessageBuilder builder; + RETURN_NOT_OK(builder.SetDictionary(id, length, body_length, nodes, buffers)); + RETURN_NOT_OK(builder.Finish()); + return builder.GetBuffer(out); +} + +Status MessageBuilder::Finish() { + auto message = + flatbuf::CreateMessage(fbb_, kMetadataVersion, header_type_, header_, body_length_); + fbb_.Finish(message); + return Status::OK(); +} + +Status MessageBuilder::GetBuffer(std::shared_ptr* out) { + int32_t size = fbb_.GetSize(); + + auto result = std::make_shared(); + RETURN_NOT_OK(result->Resize(size)); + + uint8_t* dst = result->mutable_data(); + memcpy(dst, fbb_.GetBufferPointer(), size); + + *out = result; + return Status::OK(); +} + +static flatbuffers::Offset> +FileBlocksToFlatbuffer(FBB& fbb, const std::vector& blocks) { + std::vector fb_blocks; + + for (const FileBlock& block : blocks) { + fb_blocks.emplace_back(block.offset, block.metadata_length, block.body_length); + } + + return fbb.CreateVectorOfStructs(fb_blocks); +} + +Status WriteFileFooter(const Schema& schema, const std::vector& dictionaries, + const std::vector& record_batches, DictionaryMemo* dictionary_memo, + io::OutputStream* out) { + FBB fbb; + + flatbuffers::Offset fb_schema; + RETURN_NOT_OK(SchemaToFlatbuffer(fbb, schema, dictionary_memo, &fb_schema)); + + auto fb_dictionaries = FileBlocksToFlatbuffer(fbb, dictionaries); + auto fb_record_batches = FileBlocksToFlatbuffer(fbb, record_batches); + + auto footer = flatbuf::CreateFooter( + fbb, kMetadataVersion, fb_schema, fb_dictionaries, fb_record_batches); + + fbb.Finish(footer); + + int32_t size = fbb.GetSize(); + + return out->Write(fbb.GetBufferPointer(), size); +} + // ---------------------------------------------------------------------- // Memoization data structure for handling shared dictionaries diff --git a/cpp/src/arrow/ipc/metadata.h b/cpp/src/arrow/ipc/metadata.h index a378af7b9c5..99910d1f9b4 100644 --- a/cpp/src/arrow/ipc/metadata.h +++ b/cpp/src/arrow/ipc/metadata.h @@ -221,6 +221,28 @@ class ARROW_EXPORT Message { Status ReadMessage(int64_t offset, int32_t metadata_length, io::RandomAccessFile* file, std::shared_ptr* message); +// Serialize arrow::Schema as a Flatbuffer +// +// \param[in] schema a Schema instance +// \param[inout] dictionary_memo class for tracking dictionaries and assigning +// dictionary ids +// \param[out] out the serialized arrow::Buffer +// \return Status outcome +Status WriteSchemaMessage( + const Schema& schema, DictionaryMemo* dictionary_memo, std::shared_ptr* out); + +Status WriteRecordBatchMessage(int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + std::shared_ptr* out); + +Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + std::shared_ptr* out); + +Status WriteFileFooter(const Schema& schema, const std::vector& dictionaries, + const std::vector& record_batches, DictionaryMemo* dictionary_memo, + io::OutputStream* out); + } // namespace ipc } // namespace arrow diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc index 813b82a7dbe..be88f72d64f 100644 --- a/cpp/src/arrow/ipc/reader.cc +++ b/cpp/src/arrow/ipc/reader.cc @@ -26,7 +26,8 @@ #include "arrow/buffer.h" #include "arrow/io/interfaces.h" #include "arrow/io/memory.h" -#include "arrow/ipc/metadata-internal.h" +#include "arrow/ipc/File_generated.h" +#include "arrow/ipc/Message_generated.h" #include "arrow/ipc/metadata.h" #include "arrow/ipc/util.h" #include "arrow/schema.h" @@ -35,6 +36,9 @@ #include "arrow/util/logging.h" namespace arrow { + +namespace flatbuf = org::apache::arrow::flatbuf; + namespace ipc { // ---------------------------------------------------------------------- diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index 869f46181b5..d115d9bf8d8 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -26,10 +26,9 @@ #include "arrow/buffer.h" #include "arrow/io/interfaces.h" #include "arrow/io/memory.h" -#include "arrow/ipc/Message_generated.h" -#include "arrow/ipc/metadata-internal.h" #include "arrow/ipc/metadata.h" #include "arrow/ipc/util.h" +#include "arrow/loader.h" #include "arrow/memory_pool.h" #include "arrow/schema.h" #include "arrow/status.h" @@ -39,9 +38,6 @@ #include "arrow/util/logging.h" namespace arrow { - -namespace flatbuf = org::apache::arrow::flatbuf; - namespace ipc { // ---------------------------------------------------------------------- @@ -69,8 +65,8 @@ class RecordBatchWriter : public ArrayVisitor { } // push back all common elements - field_nodes_.push_back(flatbuf::FieldNode( - static_cast(arr.length()), static_cast(arr.null_count()))); + field_nodes_.emplace_back(arr.length(), arr.null_count(), 0); + if (arr.null_count() > 0) { std::shared_ptr bitmap = arr.null_bitmap(); @@ -104,6 +100,10 @@ class RecordBatchWriter : public ArrayVisitor { // reference. May be 0 or some other position in an address space int64_t offset = buffer_start_offset_; + buffer_meta_.reserve(buffers_.size()); + + const int32_t kNoPageId = -1; + // Construct the buffer metadata for the record batch header for (size_t i = 0; i < buffers_.size(); ++i) { const Buffer* buffer = buffers_[i].get(); @@ -124,7 +124,7 @@ class RecordBatchWriter : public ArrayVisitor { // are using from any OS-level shared memory. The thought is that systems // may (in the future) associate integer page id's with physical memory // pages (according to whatever is the desired shared memory mechanism) - buffer_meta_.push_back(flatbuf::Buffer(-1, offset, size + padding)); + buffer_meta_.push_back({kNoPageId, offset, size + padding}); offset += size + padding; } @@ -458,8 +458,8 @@ class RecordBatchWriter : public ArrayVisitor { // In some cases, intermediate buffers may need to be allocated (with sliced arrays) MemoryPool* pool_; - std::vector field_nodes_; - std::vector buffer_meta_; + std::vector field_nodes_; + std::vector buffer_meta_; std::vector> buffers_; int64_t max_recursion_depth_; @@ -675,38 +675,6 @@ Status StreamWriter::Close() { // ---------------------------------------------------------------------- // File writer implementation -static flatbuffers::Offset> -FileBlocksToFlatbuffer(FBB& fbb, const std::vector& blocks) { - std::vector fb_blocks; - - for (const FileBlock& block : blocks) { - fb_blocks.emplace_back(block.offset, block.metadata_length, block.body_length); - } - - return fbb.CreateVectorOfStructs(fb_blocks); -} - -Status WriteFileFooter(const Schema& schema, const std::vector& dictionaries, - const std::vector& record_batches, DictionaryMemo* dictionary_memo, - io::OutputStream* out) { - FBB fbb; - - flatbuffers::Offset fb_schema; - RETURN_NOT_OK(SchemaToFlatbuffer(fbb, schema, dictionary_memo, &fb_schema)); - - auto fb_dictionaries = FileBlocksToFlatbuffer(fbb, dictionaries); - auto fb_record_batches = FileBlocksToFlatbuffer(fbb, record_batches); - - auto footer = flatbuf::CreateFooter( - fbb, kMetadataVersion, fb_schema, fb_dictionaries, fb_record_batches); - - fbb.Finish(footer); - - int32_t size = fbb.GetSize(); - - return out->Write(fbb.GetBufferPointer(), size); -} - class FileWriter::FileWriterImpl : public StreamWriter::StreamWriterImpl { public: using BASE = StreamWriter::StreamWriterImpl; diff --git a/cpp/src/arrow/ipc/writer.h b/cpp/src/arrow/ipc/writer.h index c1d70b5dca4..145f09d2a5a 100644 --- a/cpp/src/arrow/ipc/writer.h +++ b/cpp/src/arrow/ipc/writer.h @@ -69,6 +69,12 @@ Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); +/// EXPERIMENTAL: Write record batch using LargeRecordBatch IPC metadata. This +/// data may not be readable by all Arrow implementations +Status WriteLargeRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, + io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, + MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); + // Write Array as a DictionaryBatch message Status WriteDictionary(int64_t dictionary_id, const std::shared_ptr& dictionary, int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, @@ -102,10 +108,6 @@ class ARROW_EXPORT StreamWriter { std::unique_ptr impl_; }; -Status WriteFileFooter(const Schema& schema, const std::vector& dictionaries, - const std::vector& record_batches, DictionaryMemo* dictionary_memo, - io::OutputStream* out); - class ARROW_EXPORT FileWriter : public StreamWriter { public: static Status Open(io::OutputStream* sink, const std::shared_ptr& schema, diff --git a/cpp/src/arrow/loader.h b/cpp/src/arrow/loader.h index f116d64f5c0..9b650e2da74 100644 --- a/cpp/src/arrow/loader.h +++ b/cpp/src/arrow/loader.h @@ -41,11 +41,36 @@ struct DataType; constexpr int kMaxNestingDepth = 64; struct ARROW_EXPORT FieldMetadata { + FieldMetadata() {} + FieldMetadata(int64_t length, int64_t null_count, int64_t offset) + : length(length), null_count(null_count), offset(offset) {} + + FieldMetadata(const FieldMetadata& other) { + this->length = other.length; + this->null_count = other.null_count; + this->offset = other.offset; + } + int64_t length; int64_t null_count; int64_t offset; }; +struct ARROW_EXPORT BufferMetadata { + BufferMetadata() {} + BufferMetadata(int32_t page, int64_t offset, int64_t length) + : page(page), offset(offset), length(length) {} + + /// The shared memory page id where to find this. Set to -1 if unused + int32_t page; + + /// The relative offset into the memory page to the starting byte of the buffer + int64_t offset; + + /// Absolute length in bytes of the buffer + int64_t length; +}; + /// Implement this to create new types of Arrow data loaders class ARROW_EXPORT ArrayComponentSource { public: diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h index a143d79013f..adc3161e955 100644 --- a/cpp/src/arrow/type.h +++ b/cpp/src/arrow/type.h @@ -222,6 +222,7 @@ struct ARROW_EXPORT Field { std::string ToString() const; }; + typedef std::shared_ptr FieldPtr; struct ARROW_EXPORT PrimitiveCType : public FixedWidthType { From 85d1a1c0154da5d5f7bb61c3ceeed7cb34ceda8f Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 00:53:33 -0400 Subject: [PATCH 3/9] Add (untested) metadata writer for LargeRecordBatch Change-Id: I8fed2c84cc2e1c38ea04590487f980efacf306c4 --- cpp/src/arrow/ipc/metadata.cc | 202 ++++++++++++++++++---------------- cpp/src/arrow/ipc/metadata.h | 4 + cpp/src/arrow/ipc/writer.cc | 26 +++-- 3 files changed, 132 insertions(+), 100 deletions(-) diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc index a1f54f07ba4..70100b766c0 100644 --- a/cpp/src/arrow/ipc/metadata.cc +++ b/cpp/src/arrow/ipc/metadata.cc @@ -42,6 +42,7 @@ namespace ipc { using FBB = flatbuffers::FlatBufferBuilder; using DictionaryOffset = flatbuffers::Offset; using FieldOffset = flatbuffers::Offset; +using LargeRecordBatchOffset = flatbuffers::Offset; using RecordBatchOffset = flatbuffers::Offset; using VectorLayoutOffset = flatbuffers::Offset; using Offset = flatbuffers::Offset; @@ -480,8 +481,6 @@ static Status FieldFromFlatbufferDictionary( return Status::OK(); } -// Implement MessageBuilder - // will return the endianness of the system we are running on // based the NUMPY_API function. See NOTICE.txt flatbuf::Endianness endianness() { @@ -507,123 +506,142 @@ static Status SchemaToFlatbuffer(FBB& fbb, const Schema& schema, return Status::OK(); } -class MessageBuilder { - public: - Status SetSchema(const Schema& schema, DictionaryMemo* dictionary_memo) { - flatbuffers::Offset fb_schema; - RETURN_NOT_OK(SchemaToFlatbuffer(fbb_, schema, dictionary_memo, &fb_schema)); +static Status WriteFlatbufferBuilder(FBB& fbb, std::shared_ptr* out) { + int32_t size = fbb.GetSize(); - header_type_ = flatbuf::MessageHeader_Schema; - header_ = fb_schema.Union(); - body_length_ = 0; - return Status::OK(); - } + auto result = std::make_shared(); + RETURN_NOT_OK(result->Resize(size)); - Status MakeRecordBatch(int32_t length, int64_t body_length, - const std::vector& nodes, const std::vector& buffers, - RecordBatchOffset* offset) { - std::vector fb_nodes; - std::vector fb_buffers; - fb_nodes.reserve(nodes.size()); - fb_buffers.reserve(buffers.size()); + uint8_t* dst = result->mutable_data(); + memcpy(dst, fbb.GetBufferPointer(), size); + *out = result; + return Status::OK(); +} - for (size_t i = 0; i < nodes.size(); ++i) { - const FieldMetadata& node = nodes[i]; - if (node.offset != 0) { - return Status::Invalid("Field metadata for IPC must have offset 0"); - } - fb_nodes.emplace_back(node.length, node.null_count); - } +static Status WriteMessage(FBB& fbb, flatbuf::MessageHeader header_type, + flatbuffers::Offset header, int64_t body_length, std::shared_ptr* out) { + auto message = + flatbuf::CreateMessage(fbb, kMetadataVersion, header_type, header, body_length); + fbb.Finish(message); + return WriteFlatbufferBuilder(fbb, out); +} - for (size_t i = 0; i < buffers.size(); ++i) { - const BufferMetadata& buffer = buffers[i]; - fb_buffers.emplace_back(buffer.page, buffer.offset, buffer.length); +Status WriteSchemaMessage( + const Schema& schema, DictionaryMemo* dictionary_memo, std::shared_ptr* out) { + FBB fbb; + flatbuffers::Offset fb_schema; + RETURN_NOT_OK(SchemaToFlatbuffer(fbb, schema, dictionary_memo, &fb_schema)); + return WriteMessage(fbb, flatbuf::MessageHeader_Schema, fb_schema.Union(), 0, out); +} + +using FieldNodeVector = + flatbuffers::Offset>; +using LargeFieldNodeVector = + flatbuffers::Offset>; +using BufferVector = flatbuffers::Offset>; + +static Status WriteFieldNodes( + FBB& fbb, const std::vector& nodes, FieldNodeVector* out) { + std::vector fb_nodes; + fb_nodes.reserve(nodes.size()); + + for (size_t i = 0; i < nodes.size(); ++i) { + const FieldMetadata& node = nodes[i]; + if (node.offset != 0) { + return Status::Invalid("Field metadata for IPC must have offset 0"); } - *offset = flatbuf::CreateRecordBatch(fbb_, length, - fbb_.CreateVectorOfStructs(fb_nodes), fbb_.CreateVectorOfStructs(fb_buffers)); - return Status::OK(); + fb_nodes.emplace_back( + static_cast(node.length), static_cast(node.null_count)); } + *out = fbb.CreateVectorOfStructs(fb_nodes); + return Status::OK(); +} - Status SetRecordBatch(int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers) { - header_type_ = flatbuf::MessageHeader_RecordBatch; - - RecordBatchOffset batch_offset; - RETURN_NOT_OK(MakeRecordBatch(length, body_length, nodes, buffers, &batch_offset)); - header_ = batch_offset.Union(); - body_length_ = body_length; +static Status WriteLargeFieldNodes( + FBB& fbb, const std::vector& nodes, LargeFieldNodeVector* out) { + std::vector fb_nodes; + fb_nodes.reserve(nodes.size()); - return Status::OK(); + for (size_t i = 0; i < nodes.size(); ++i) { + const FieldMetadata& node = nodes[i]; + if (node.offset != 0) { + return Status::Invalid("Field metadata for IPC must have offset 0"); + } + fb_nodes.emplace_back(node.length, node.null_count); } + *out = fbb.CreateVectorOfStructs(fb_nodes); + return Status::OK(); +} - Status SetDictionary(int64_t id, int32_t length, int64_t body_length, - const std::vector& nodes, - const std::vector& buffers) { - header_type_ = flatbuf::MessageHeader_DictionaryBatch; +static Status WriteBuffers( + FBB& fbb, const std::vector& buffers, BufferVector* out) { + std::vector fb_buffers; + fb_buffers.reserve(buffers.size()); - RecordBatchOffset record_batch; - RETURN_NOT_OK(MakeRecordBatch(length, body_length, nodes, buffers, &record_batch)); - header_ = flatbuf::CreateDictionaryBatch(fbb_, id, record_batch).Union(); - body_length_ = body_length; - return Status::OK(); + for (size_t i = 0; i < buffers.size(); ++i) { + const BufferMetadata& buffer = buffers[i]; + fb_buffers.emplace_back(buffer.page, buffer.offset, buffer.length); } + *out = fbb.CreateVectorOfStructs(fb_buffers); + return Status::OK(); +} - Status Finish(); +static Status MakeRecordBatch(FBB& fbb, int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + RecordBatchOffset* offset) { + FieldNodeVector fb_nodes; + BufferVector fb_buffers; - Status GetBuffer(std::shared_ptr* out); + RETURN_NOT_OK(WriteFieldNodes(fbb, nodes, &fb_nodes)); + RETURN_NOT_OK(WriteBuffers(fbb, buffers, &fb_buffers)); - private: - flatbuf::MessageHeader header_type_; - flatbuffers::Offset header_; - int64_t body_length_; - flatbuffers::FlatBufferBuilder fbb_; -}; + *offset = flatbuf::CreateRecordBatch(fbb, length, fb_nodes, fb_buffers); + return Status::OK(); +} -Status WriteSchemaMessage( - const Schema& schema, DictionaryMemo* dictionary_memo, std::shared_ptr* out) { - MessageBuilder message; - RETURN_NOT_OK(message.SetSchema(schema, dictionary_memo)); - RETURN_NOT_OK(message.Finish()); - return message.GetBuffer(out); +static Status MakeLargeRecordBatch(FBB& fbb, int64_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + LargeRecordBatchOffset* offset) { + LargeFieldNodeVector fb_nodes; + BufferVector fb_buffers; + + RETURN_NOT_OK(WriteLargeFieldNodes(fbb, nodes, &fb_nodes)); + RETURN_NOT_OK(WriteBuffers(fbb, buffers, &fb_buffers)); + + *offset = flatbuf::CreateLargeRecordBatch(fbb, length, fb_nodes, fb_buffers); + return Status::OK(); } Status WriteRecordBatchMessage(int32_t length, int64_t body_length, const std::vector& nodes, const std::vector& buffers, std::shared_ptr* out) { - MessageBuilder builder; - RETURN_NOT_OK(builder.SetRecordBatch(length, body_length, nodes, buffers)); - RETURN_NOT_OK(builder.Finish()); - return builder.GetBuffer(out); + FBB fbb; + RecordBatchOffset record_batch; + RETURN_NOT_OK(MakeRecordBatch(fbb, length, body_length, nodes, buffers, &record_batch)); + return WriteMessage( + fbb, flatbuf::MessageHeader_RecordBatch, record_batch.Union(), body_length, out); } -Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, +Status WriteLargeRecordBatchMessage(int64_t length, int64_t body_length, const std::vector& nodes, const std::vector& buffers, std::shared_ptr* out) { - MessageBuilder builder; - RETURN_NOT_OK(builder.SetDictionary(id, length, body_length, nodes, buffers)); - RETURN_NOT_OK(builder.Finish()); - return builder.GetBuffer(out); -} - -Status MessageBuilder::Finish() { - auto message = - flatbuf::CreateMessage(fbb_, kMetadataVersion, header_type_, header_, body_length_); - fbb_.Finish(message); - return Status::OK(); + FBB fbb; + LargeRecordBatchOffset record_batch; + RETURN_NOT_OK( + MakeLargeRecordBatch(fbb, length, body_length, nodes, buffers, &record_batch)); + return WriteMessage( + fbb, flatbuf::MessageHeader_RecordBatch, record_batch.Union(), body_length, out); } -Status MessageBuilder::GetBuffer(std::shared_ptr* out) { - int32_t size = fbb_.GetSize(); - - auto result = std::make_shared(); - RETURN_NOT_OK(result->Resize(size)); - - uint8_t* dst = result->mutable_data(); - memcpy(dst, fbb_.GetBufferPointer(), size); - - *out = result; - return Status::OK(); +Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + std::shared_ptr* out) { + FBB fbb; + RecordBatchOffset record_batch; + RETURN_NOT_OK(MakeRecordBatch(fbb, length, body_length, nodes, buffers, &record_batch)); + auto dictionary_batch = flatbuf::CreateDictionaryBatch(fbb, id, record_batch).Union(); + return WriteMessage( + fbb, flatbuf::MessageHeader_DictionaryBatch, dictionary_batch, body_length, out); } static flatbuffers::Offset> diff --git a/cpp/src/arrow/ipc/metadata.h b/cpp/src/arrow/ipc/metadata.h index 99910d1f9b4..6d8eae15a89 100644 --- a/cpp/src/arrow/ipc/metadata.h +++ b/cpp/src/arrow/ipc/metadata.h @@ -235,6 +235,10 @@ Status WriteRecordBatchMessage(int32_t length, int64_t body_length, const std::vector& nodes, const std::vector& buffers, std::shared_ptr* out); +Status WriteLargeRecordBatchMessage(int64_t length, int64_t body_length, + const std::vector& nodes, const std::vector& buffers, + std::shared_ptr* out); + Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, const std::vector& nodes, const std::vector& buffers, std::shared_ptr* out); diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index d115d9bf8d8..ae1da71e2e9 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -136,12 +136,12 @@ class RecordBatchWriter : public ArrayVisitor { // Override this for writing dictionary metadata virtual Status WriteMetadataMessage( - int32_t num_rows, int64_t body_length, std::shared_ptr* out) { + int64_t num_rows, int64_t body_length, std::shared_ptr* out) { return WriteRecordBatchMessage( - num_rows, body_length, field_nodes_, buffer_meta_, out); + static_cast(num_rows), body_length, field_nodes_, buffer_meta_, out); } - Status WriteMetadata(int32_t num_rows, int64_t body_length, io::OutputStream* dst, + Status WriteMetadata(int64_t num_rows, int64_t body_length, io::OutputStream* dst, int32_t* metadata_length) { // Now that we have computed the locations of all of the buffers in shared // memory, the data header can be converted to a flatbuffer and written out @@ -190,8 +190,7 @@ class RecordBatchWriter : public ArrayVisitor { RETURN_NOT_OK(dst->Tell(&start_position)); #endif - RETURN_NOT_OK(WriteMetadata( - static_cast(batch.num_rows()), *body_length, dst, metadata_length)); + RETURN_NOT_OK(WriteMetadata(batch.num_rows(), *body_length, dst, metadata_length)); #ifndef NDEBUG RETURN_NOT_OK(dst->Tell(¤t_position)); @@ -466,14 +465,25 @@ class RecordBatchWriter : public ArrayVisitor { int64_t buffer_start_offset_; }; +class LargeRecordBatchWriter : public RecordBatchWriter { + public: + using RecordBatchWriter::RecordBatchWriter; + + Status WriteMetadataMessage( + int64_t num_rows, int64_t body_length, std::shared_ptr* out) override { + return WriteLargeRecordBatchMessage( + num_rows, body_length, field_nodes_, buffer_meta_, out); + } +}; + class DictionaryWriter : public RecordBatchWriter { public: using RecordBatchWriter::RecordBatchWriter; Status WriteMetadataMessage( - int32_t num_rows, int64_t body_length, std::shared_ptr* out) override { - return WriteDictionaryMessage( - dictionary_id_, num_rows, body_length, field_nodes_, buffer_meta_, out); + int64_t num_rows, int64_t body_length, std::shared_ptr* out) override { + return WriteDictionaryMessage(dictionary_id_, static_cast(num_rows), + body_length, field_nodes_, buffer_meta_, out); } Status Write(int64_t dictionary_id, const std::shared_ptr& dictionary, From f4c883060bfc048c662652e501c6a05beaa733fd Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 15:42:06 -0400 Subject: [PATCH 4/9] Consolidate ipc-metadata-test and ipc-read-write-test and draft large record batch read/write path Change-Id: I29604a1ed32e8598d39a2ceb632578e5fb8512b8 --- cpp/src/arrow/ipc/CMakeLists.txt | 4 - cpp/src/arrow/ipc/ipc-metadata-test.cc | 99 ------------------- cpp/src/arrow/ipc/ipc-read-write-test.cc | 60 ++++++++++++ cpp/src/arrow/ipc/metadata.cc | 54 ++++++----- cpp/src/arrow/ipc/metadata.h | 10 +- cpp/src/arrow/ipc/reader.cc | 116 +++++++++++++++-------- cpp/src/arrow/ipc/reader.h | 8 ++ cpp/src/arrow/ipc/writer.cc | 48 +++++++--- cpp/src/arrow/ipc/writer.h | 14 +-- format/Message.fbs | 4 +- 10 files changed, 223 insertions(+), 194 deletions(-) delete mode 100644 cpp/src/arrow/ipc/ipc-metadata-test.cc diff --git a/cpp/src/arrow/ipc/CMakeLists.txt b/cpp/src/arrow/ipc/CMakeLists.txt index 9ea0eebcb27..5d470df0309 100644 --- a/cpp/src/arrow/ipc/CMakeLists.txt +++ b/cpp/src/arrow/ipc/CMakeLists.txt @@ -66,10 +66,6 @@ ADD_ARROW_TEST(ipc-read-write-test) ARROW_TEST_LINK_LIBRARIES(ipc-read-write-test ${ARROW_IPC_TEST_LINK_LIBS}) -ADD_ARROW_TEST(ipc-metadata-test) -ARROW_TEST_LINK_LIBRARIES(ipc-metadata-test - ${ARROW_IPC_TEST_LINK_LIBS}) - ADD_ARROW_TEST(ipc-json-test) ARROW_TEST_LINK_LIBRARIES(ipc-json-test ${ARROW_IPC_TEST_LINK_LIBS}) diff --git a/cpp/src/arrow/ipc/ipc-metadata-test.cc b/cpp/src/arrow/ipc/ipc-metadata-test.cc deleted file mode 100644 index e98e25043cd..00000000000 --- a/cpp/src/arrow/ipc/ipc-metadata-test.cc +++ /dev/null @@ -1,99 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include -#include -#include - -#include "gtest/gtest.h" - -#include "arrow/io/memory.h" -#include "arrow/ipc/metadata.h" -#include "arrow/ipc/test-common.h" -#include "arrow/schema.h" -#include "arrow/status.h" -#include "arrow/test-util.h" -#include "arrow/type.h" - -namespace arrow { - -class Buffer; - -namespace ipc { - -class TestSchemaMetadata : public ::testing::Test { - public: - void SetUp() {} - - void CheckRoundtrip(const Schema& schema, DictionaryMemo* memo) { - std::shared_ptr buffer; - ASSERT_OK(WriteSchemaMessage(schema, memo, &buffer)); - - std::shared_ptr message; - ASSERT_OK(Message::Open(buffer, 0, &message)); - - ASSERT_EQ(Message::SCHEMA, message->type()); - - auto schema_msg = std::make_shared(message); - ASSERT_EQ(schema.num_fields(), schema_msg->num_fields()); - - DictionaryMemo empty_memo; - - std::shared_ptr schema2; - ASSERT_OK(schema_msg->GetSchema(empty_memo, &schema2)); - - AssertSchemaEqual(schema, *schema2); - } -}; - -const std::shared_ptr INT32 = std::make_shared(); - -TEST_F(TestSchemaMetadata, PrimitiveFields) { - auto f0 = std::make_shared("f0", std::make_shared()); - auto f1 = std::make_shared("f1", std::make_shared(), false); - auto f2 = std::make_shared("f2", std::make_shared()); - auto f3 = std::make_shared("f3", std::make_shared()); - auto f4 = std::make_shared("f4", std::make_shared()); - auto f5 = std::make_shared("f5", std::make_shared()); - auto f6 = std::make_shared("f6", std::make_shared()); - auto f7 = std::make_shared("f7", std::make_shared()); - auto f8 = std::make_shared("f8", std::make_shared()); - auto f9 = std::make_shared("f9", std::make_shared(), false); - auto f10 = std::make_shared("f10", std::make_shared()); - - Schema schema({f0, f1, f2, f3, f4, f5, f6, f7, f8, f9, f10}); - DictionaryMemo memo; - - CheckRoundtrip(schema, &memo); -} - -TEST_F(TestSchemaMetadata, NestedFields) { - auto type = std::make_shared(std::make_shared()); - auto f0 = std::make_shared("f0", type); - - std::shared_ptr type2(new StructType({std::make_shared("k1", INT32), - std::make_shared("k2", INT32), std::make_shared("k3", INT32)})); - auto f1 = std::make_shared("f1", type2); - - Schema schema({f0, f1}); - DictionaryMemo memo; - - CheckRoundtrip(schema, &memo); -} - -} // namespace ipc -} // namespace arrow diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc index d2cc7b1cf60..c34750a18a4 100644 --- a/cpp/src/arrow/ipc/ipc-read-write-test.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc @@ -57,6 +57,66 @@ void CompareBatch(const RecordBatch& left, const RecordBatch& right) { using BatchVector = std::vector>; +class TestSchemaMetadata : public ::testing::Test { + public: + void SetUp() {} + + void CheckRoundtrip(const Schema& schema, DictionaryMemo* memo) { + std::shared_ptr buffer; + ASSERT_OK(WriteSchemaMessage(schema, memo, &buffer)); + + std::shared_ptr message; + ASSERT_OK(Message::Open(buffer, 0, &message)); + + ASSERT_EQ(Message::SCHEMA, message->type()); + + auto schema_msg = std::make_shared(message); + ASSERT_EQ(schema.num_fields(), schema_msg->num_fields()); + + DictionaryMemo empty_memo; + + std::shared_ptr schema2; + ASSERT_OK(schema_msg->GetSchema(empty_memo, &schema2)); + + AssertSchemaEqual(schema, *schema2); + } +}; + +const std::shared_ptr INT32 = std::make_shared(); + +TEST_F(TestSchemaMetadata, PrimitiveFields) { + auto f0 = std::make_shared("f0", std::make_shared()); + auto f1 = std::make_shared("f1", std::make_shared(), false); + auto f2 = std::make_shared("f2", std::make_shared()); + auto f3 = std::make_shared("f3", std::make_shared()); + auto f4 = std::make_shared("f4", std::make_shared()); + auto f5 = std::make_shared("f5", std::make_shared()); + auto f6 = std::make_shared("f6", std::make_shared()); + auto f7 = std::make_shared("f7", std::make_shared()); + auto f8 = std::make_shared("f8", std::make_shared()); + auto f9 = std::make_shared("f9", std::make_shared(), false); + auto f10 = std::make_shared("f10", std::make_shared()); + + Schema schema({f0, f1, f2, f3, f4, f5, f6, f7, f8, f9, f10}); + DictionaryMemo memo; + + CheckRoundtrip(schema, &memo); +} + +TEST_F(TestSchemaMetadata, NestedFields) { + auto type = std::make_shared(std::make_shared()); + auto f0 = std::make_shared("f0", type); + + std::shared_ptr type2(new StructType({std::make_shared("k1", INT32), + std::make_shared("k2", INT32), std::make_shared("k3", INT32)})); + auto f1 = std::make_shared("f1", type2); + + Schema schema({f0, f1}); + DictionaryMemo memo; + + CheckRoundtrip(schema, &memo); +} + #define BATCH_CASES() \ ::testing::Values(&MakeIntRecordBatch, &MakeListRecordBatch, &MakeNonNullRecordBatch, \ &MakeZeroLengthRecordBatch, &MakeDeeplyNestedList, &MakeStringTypesRecordBatch, \ diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc index 70100b766c0..a3605809424 100644 --- a/cpp/src/arrow/ipc/metadata.cc +++ b/cpp/src/arrow/ipc/metadata.cc @@ -626,11 +626,12 @@ Status WriteLargeRecordBatchMessage(int64_t length, int64_t body_length, const std::vector& nodes, const std::vector& buffers, std::shared_ptr* out) { FBB fbb; - LargeRecordBatchOffset record_batch; + LargeRecordBatchOffset large_batch; RETURN_NOT_OK( - MakeLargeRecordBatch(fbb, length, body_length, nodes, buffers, &record_batch)); - return WriteMessage( - fbb, flatbuf::MessageHeader_RecordBatch, record_batch.Union(), body_length, out); + MakeLargeRecordBatch(fbb, length, body_length, nodes, buffers, &large_batch)); + + fbb.Finish(large_batch); + return WriteFlatbufferBuilder(fbb, out); } Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, @@ -795,7 +796,18 @@ int64_t Message::body_length() const { // ---------------------------------------------------------------------- // SchemaMetadata -class SchemaMetadata::SchemaMetadataImpl { +class MessageHolder { + public: + void set_message(const std::shared_ptr& message) { message_ = message; } + void set_buffer(const std::shared_ptr& buffer) { buffer_ = buffer; } + + protected: + // Possible parents, owns the flatbuffer data + std::shared_ptr message_; + std::shared_ptr buffer_; +}; + +class SchemaMetadata::SchemaMetadataImpl : public MessageHolder { public: explicit SchemaMetadataImpl(const void* schema) : schema_(static_cast(schema)) {} @@ -833,15 +845,19 @@ class SchemaMetadata::SchemaMetadataImpl { const flatbuf::Schema* schema_; }; -SchemaMetadata::SchemaMetadata( - const std::shared_ptr& message, const void* flatbuf) { - message_ = message; - impl_.reset(new SchemaMetadataImpl(flatbuf)); +SchemaMetadata::SchemaMetadata(const std::shared_ptr& message) + : SchemaMetadata(message->impl_->header()) { + impl_->set_message(message); } -SchemaMetadata::SchemaMetadata(const std::shared_ptr& message) { - message_ = message; - impl_.reset(new SchemaMetadataImpl(message->impl_->header())); +SchemaMetadata::SchemaMetadata(const void* header) { + impl_.reset(new SchemaMetadataImpl(header)); +} + +SchemaMetadata::SchemaMetadata(const std::shared_ptr& buffer, int64_t offset) + : SchemaMetadata(buffer->data() + offset) { + // Preserve ownership + impl_->set_buffer(buffer); } SchemaMetadata::~SchemaMetadata() {} @@ -868,7 +884,7 @@ Status SchemaMetadata::GetSchema( // ---------------------------------------------------------------------- // RecordBatchMetadata -class RecordBatchMetadata::RecordBatchMetadataImpl { +class RecordBatchMetadata::RecordBatchMetadataImpl : public MessageHolder { public: explicit RecordBatchMetadataImpl(const void* batch) : batch_(static_cast(batch)) { @@ -886,22 +902,14 @@ class RecordBatchMetadata::RecordBatchMetadataImpl { int num_fields() const { return batch_->nodes()->size(); } - void set_message(const std::shared_ptr& message) { message_ = message; } - - void set_buffer(const std::shared_ptr& buffer) { buffer_ = buffer; } - private: const flatbuf::RecordBatch* batch_; const flatbuffers::Vector* nodes_; const flatbuffers::Vector* buffers_; - - // Possible parents, owns the flatbuffer data - std::shared_ptr message_; - std::shared_ptr buffer_; }; -RecordBatchMetadata::RecordBatchMetadata(const std::shared_ptr& message) { - impl_.reset(new RecordBatchMetadataImpl(message->impl_->header())); +RecordBatchMetadata::RecordBatchMetadata(const std::shared_ptr& message) + : RecordBatchMetadata(message->impl_->header()) { impl_->set_message(message); } diff --git a/cpp/src/arrow/ipc/metadata.h b/cpp/src/arrow/ipc/metadata.h index 6d8eae15a89..41e6c5e9f19 100644 --- a/cpp/src/arrow/ipc/metadata.h +++ b/cpp/src/arrow/ipc/metadata.h @@ -107,10 +107,9 @@ class Message; // Container for serialized Schema metadata contained in an IPC message class ARROW_EXPORT SchemaMetadata { public: + explicit SchemaMetadata(const void* header); explicit SchemaMetadata(const std::shared_ptr& message); - - // Accepts an opaque flatbuffer pointer - SchemaMetadata(const std::shared_ptr& message, const void* schema); + SchemaMetadata(const std::shared_ptr& message, int64_t offset); ~SchemaMetadata(); @@ -127,9 +126,6 @@ class ARROW_EXPORT SchemaMetadata { const DictionaryMemo& dictionary_memo, std::shared_ptr* out) const; private: - // Parent, owns the flatbuffer data - std::shared_ptr message_; - class SchemaMetadataImpl; std::unique_ptr impl_; @@ -145,8 +141,6 @@ struct ARROW_EXPORT BufferMetadata { // Container for serialized record batch metadata contained in an IPC message class ARROW_EXPORT RecordBatchMetadata { public: - // Instantiate from opaque pointer. Memory ownership must be preserved - // elsewhere (e.g. in a dictionary batch) explicit RecordBatchMetadata(const void* header); explicit RecordBatchMetadata(const std::shared_ptr& message); RecordBatchMetadata(const std::shared_ptr& message, int64_t offset); diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc index be88f72d64f..606d382c729 100644 --- a/cpp/src/arrow/ipc/reader.cc +++ b/cpp/src/arrow/ipc/reader.cc @@ -73,52 +73,37 @@ class IpcComponentSource : public ArrayComponentSource { io::RandomAccessFile* file_; }; -class RecordBatchReader { - public: - RecordBatchReader(const RecordBatchMetadata& metadata, - const std::shared_ptr& schema, int max_recursion_depth, - io::RandomAccessFile* file) - : metadata_(metadata), - schema_(schema), - max_recursion_depth_(max_recursion_depth), - file_(file) {} - - Status Read(std::shared_ptr* out) { - std::vector> arrays(schema_->num_fields()); - - IpcComponentSource source(metadata_, file_); - ArrayLoaderContext context; - context.source = &source; - context.field_index = 0; - context.buffer_index = 0; - context.max_recursion_depth = max_recursion_depth_; - - for (int i = 0; i < schema_->num_fields(); ++i) { - RETURN_NOT_OK(LoadArray(schema_->field(i)->type, &context, &arrays[i])); - } - - *out = std::make_shared(schema_, metadata_.length(), arrays); - return Status::OK(); - } - - private: - const RecordBatchMetadata& metadata_; - std::shared_ptr schema_; - int max_recursion_depth_; - io::RandomAccessFile* file_; -}; - Status ReadRecordBatch(const RecordBatchMetadata& metadata, const std::shared_ptr& schema, io::RandomAccessFile* file, std::shared_ptr* out) { return ReadRecordBatch(metadata, schema, kMaxNestingDepth, file, out); } +static Status LoadRecordBatchFromSource(const std::shared_ptr& schema, + int64_t num_rows, int max_recursion_depth, ArrayComponentSource* source, + std::shared_ptr* out) { + std::vector> arrays(schema->num_fields()); + + ArrayLoaderContext context; + context.source = source; + context.field_index = 0; + context.buffer_index = 0; + context.max_recursion_depth = max_recursion_depth; + + for (int i = 0; i < schema->num_fields(); ++i) { + RETURN_NOT_OK(LoadArray(schema->field(i)->type, &context, &arrays[i])); + } + + *out = std::make_shared(schema, num_rows, arrays); + return Status::OK(); +} + Status ReadRecordBatch(const RecordBatchMetadata& metadata, const std::shared_ptr& schema, int max_recursion_depth, io::RandomAccessFile* file, std::shared_ptr* out) { - RecordBatchReader reader(metadata, schema, max_recursion_depth, file); - return reader.Read(out); + IpcComponentSource source(metadata, file); + return LoadRecordBatchFromSource( + schema, metadata.length(), max_recursion_depth, &source, out); } Status ReadDictionary(const DictionaryBatchMetadata& metadata, @@ -341,7 +326,7 @@ class FileReader::FileReaderImpl { // TODO(wesm): Verify the footer footer_ = flatbuf::GetFooter(footer_buffer_->data()); - schema_metadata_.reset(new SchemaMetadata(nullptr, footer_->schema())); + schema_metadata_.reset(new SchemaMetadata(footer_->schema())); return Status::OK(); } @@ -483,5 +468,60 @@ Status FileReader::GetRecordBatch(int i, std::shared_ptr* batch) { return impl_->GetRecordBatch(i, batch); } +// ---------------------------------------------------------------------- +// Read LargeRecordBatch + +class LargeRecordBatchSource : public ArrayComponentSource { + public: + LargeRecordBatchSource( + const flatbuf::LargeRecordBatch* metadata, io::RandomAccessFile* file) + : metadata_(metadata), file_(file) {} + + Status GetBuffer(int buffer_index, std::shared_ptr* out) override { + if (buffer_index >= static_cast(metadata_->buffers()->size())) { + return Status::Invalid("Ran out of buffer metadata, likely malformed"); + } + const flatbuf::Buffer* buffer = metadata_->buffers()->Get(buffer_index); + + if (buffer->length() == 0) { + *out = nullptr; + return Status::OK(); + } else { + return file_->ReadAt(buffer->offset(), buffer->length(), out); + } + } + + Status GetFieldMetadata(int field_index, FieldMetadata* metadata) override { + // pop off a field + if (field_index >= static_cast(metadata_->nodes()->size())) { + return Status::Invalid("Ran out of field metadata, likely malformed"); + } + const flatbuf::LargeFieldNode* node = metadata_->nodes()->Get(field_index); + + metadata->length = node->length(); + metadata->null_count = node->null_count(); + metadata->offset = 0; + return Status::OK(); + } + + private: + const flatbuf::LargeRecordBatch* metadata_; + io::RandomAccessFile* file_; +}; + +Status ReadLargeRecordBatch(const std::shared_ptr& schema, int64_t offset, + io::RandomAccessFile* file, std::shared_ptr* out) { + std::shared_ptr buffer; + RETURN_NOT_OK(file->ReadAt(offset, sizeof(int32_t), &buffer)); + int32_t flatbuffer_size = *reinterpret_cast(buffer->data()); + + RETURN_NOT_OK(file->ReadAt(offset + sizeof(int32_t), flatbuffer_size, &buffer)); + + auto metadata = flatbuffers::GetRoot(buffer->data()); + LargeRecordBatchSource source(metadata, file); + return LoadRecordBatchFromSource( + schema, metadata->length(), kMaxNestingDepth, &source, out); +} + } // namespace ipc } // namespace arrow diff --git a/cpp/src/arrow/ipc/reader.h b/cpp/src/arrow/ipc/reader.h index 43ada5a7877..c6ec373eead 100644 --- a/cpp/src/arrow/ipc/reader.h +++ b/cpp/src/arrow/ipc/reader.h @@ -120,6 +120,14 @@ class ARROW_EXPORT FileReader { std::unique_ptr impl_; }; +// ---------------------------------------------------------------------- +// + +/// EXPERIMENTAL: Read length-prefixed LargeRecordBatch metadata (64-bit array +/// lengths) at offset and reconstruct RecordBatch +Status ReadLargeRecordBatch(const std::shared_ptr& schema, int64_t offset, + io::RandomAccessFile* file, std::shared_ptr* out); + } // namespace ipc } // namespace arrow diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index ae1da71e2e9..0c0d693a2cd 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -55,14 +55,19 @@ class RecordBatchWriter : public ArrayVisitor { virtual ~RecordBatchWriter() = default; + virtual Status CheckArrayMetadata(const Array& arr) { + if (arr.length() > std::numeric_limits::max()) { + return Status::Invalid("Cannot write arrays larger than 2^31 - 1 in length"); + } + return Status::OK(); + } + Status VisitArray(const Array& arr) { if (max_recursion_depth_ <= 0) { return Status::Invalid("Max recursion depth reached"); } - if (arr.length() > std::numeric_limits::max()) { - return Status::Invalid("Cannot write arrays larger than 2^31 - 1 in length"); - } + RETURN_NOT_OK(CheckArrayMetadata(arr)); // push back all common elements field_nodes_.emplace_back(arr.length(), arr.null_count(), 0); @@ -465,17 +470,6 @@ class RecordBatchWriter : public ArrayVisitor { int64_t buffer_start_offset_; }; -class LargeRecordBatchWriter : public RecordBatchWriter { - public: - using RecordBatchWriter::RecordBatchWriter; - - Status WriteMetadataMessage( - int64_t num_rows, int64_t body_length, std::shared_ptr* out) override { - return WriteLargeRecordBatchMessage( - num_rows, body_length, field_nodes_, buffer_meta_, out); - } -}; - class DictionaryWriter : public RecordBatchWriter { public: using RecordBatchWriter::RecordBatchWriter; @@ -737,5 +731,31 @@ Status FileWriter::Close() { return impl_->Close(); } +// ---------------------------------------------------------------------- +// Write record batches with 64-bit size metadata + +class LargeRecordBatchWriter : public RecordBatchWriter { + public: + using RecordBatchWriter::RecordBatchWriter; + + Status CheckArrayMetadata(const Array& arr) override { + // No < INT32_MAX length check + return Status::OK(); + } + + Status WriteMetadataMessage( + int64_t num_rows, int64_t body_length, std::shared_ptr* out) override { + return WriteLargeRecordBatchMessage( + num_rows, body_length, field_nodes_, buffer_meta_, out); + } +}; + +Status WriteLargeRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, + io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, + MemoryPool* pool, int max_recursion_depth) { + LargeRecordBatchWriter writer(pool, buffer_start_offset, max_recursion_depth); + return writer.Write(batch, dst, metadata_length, body_length); +} + } // namespace ipc } // namespace arrow diff --git a/cpp/src/arrow/ipc/writer.h b/cpp/src/arrow/ipc/writer.h index 145f09d2a5a..1271652a35c 100644 --- a/cpp/src/arrow/ipc/writer.h +++ b/cpp/src/arrow/ipc/writer.h @@ -69,12 +69,6 @@ Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); -/// EXPERIMENTAL: Write record batch using LargeRecordBatch IPC metadata. This -/// data may not be readable by all Arrow implementations -Status WriteLargeRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, - io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, - MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); - // Write Array as a DictionaryBatch message Status WriteDictionary(int64_t dictionary_id, const std::shared_ptr& dictionary, int64_t buffer_start_offset, io::OutputStream* dst, int32_t* metadata_length, @@ -122,6 +116,14 @@ class ARROW_EXPORT FileWriter : public StreamWriter { std::unique_ptr impl_; }; +// ---------------------------------------------------------------------- + +/// EXPERIMENTAL: Write record batch using LargeRecordBatch IPC metadata. This +/// data may not be readable by all Arrow implementations +Status WriteLargeRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset, + io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length, + MemoryPool* pool, int max_recursion_depth = kMaxNestingDepth); + } // namespace ipc } // namespace arrow diff --git a/format/Message.fbs b/format/Message.fbs index c54218edc49..f3379f46134 100644 --- a/format/Message.fbs +++ b/format/Message.fbs @@ -308,8 +308,8 @@ table RecordBatch { /// compliant struct LargeFieldNode { - length: int; - null_count: int; + length: long; + null_count: long; } table LargeRecordBatch { From 4c1d08cfc3e294903b1ce4d94db55bfd2ab8da84 Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 16:05:09 -0400 Subject: [PATCH 5/9] Refactoring, failing test fixture for large record batch Change-Id: I340cb6744d883c361b64d090133773c084586ccf --- cpp/src/arrow/io/test-common.h | 18 ++++++++ cpp/src/arrow/ipc/ipc-read-write-test.cc | 59 ++++++++++++++++-------- cpp/src/arrow/ipc/reader.h | 4 +- 3 files changed, 61 insertions(+), 20 deletions(-) diff --git a/cpp/src/arrow/io/test-common.h b/cpp/src/arrow/io/test-common.h index 8355714540e..4c114760e9a 100644 --- a/cpp/src/arrow/io/test-common.h +++ b/cpp/src/arrow/io/test-common.h @@ -41,6 +41,24 @@ namespace arrow { namespace io { +static inline Status ZeroMemoryMap(MemoryMappedFile* file) { + constexpr int64_t kBufferSize = 512; + static constexpr uint8_t kZeroBytes[kBufferSize] = {0}; + + RETURN_NOT_OK(file->Seek(0)); + int64_t position = 0; + int64_t file_size; + RETURN_NOT_OK(file->GetSize(&file_size)); + + int64_t chunksize; + while (position < file_size) { + chunksize = std::min(kBufferSize, file_size - position); + RETURN_NOT_OK(file->Write(kZeroBytes, chunksize)); + position += chunksize; + } + return Status::OK(); +} + class MemoryMapFixture { public: void TearDown() { diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc index c34750a18a4..2e7d98fa32d 100644 --- a/cpp/src/arrow/ipc/ipc-read-write-test.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc @@ -125,16 +125,16 @@ TEST_F(TestSchemaMetadata, NestedFields) { class IpcTestFixture : public io::MemoryMapFixture { public: - Status RoundTripHelper(const RecordBatch& batch, int memory_map_size, - std::shared_ptr* batch_result) { - std::string path = "test-write-row-batch"; - io::MemoryMapFixture::InitMemoryMap(memory_map_size, path, &mmap_); - + Status DoStandardRoundTrip( + const RecordBatch& batch, std::shared_ptr* batch_result) { int32_t metadata_length; int64_t body_length; const int64_t buffer_offset = 0; + 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_)); @@ -151,20 +151,33 @@ class IpcTestFixture : public io::MemoryMapFixture { return ReadRecordBatch(*metadata, batch.schema(), &buffer_reader, batch_result); } - void CheckRoundtrip(const RecordBatch& batch, int64_t buffer_size) { - std::shared_ptr batch_result; + Status DoLargeRoundTrip( + const RecordBatch& batch, std::shared_ptr* result) { + int32_t metadata_length; + int64_t body_length; - ASSERT_OK(RoundTripHelper(batch, 1 << 16, &batch_result)); - EXPECT_EQ(batch.num_rows(), batch_result->num_rows()); + const int64_t buffer_offset = 0; - ASSERT_TRUE(batch.schema()->Equals(batch_result->schema())); - ASSERT_EQ(batch.num_columns(), batch_result->num_columns()) - << batch.schema()->ToString() - << " result: " << batch_result->schema()->ToString(); + RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); + RETURN_NOT_OK(mmap_->Seek(0)); - for (int i = 0; i < batch.num_columns(); ++i) { - const auto& left = *batch.column(i); - const auto& right = *batch_result->column(i); + RETURN_NOT_OK(WriteLargeRecordBatch( + batch, buffer_offset, mmap_.get(), &metadata_length, &body_length, pool_)); + + return ReadLargeRecordBatch(batch.schema(), 0, mmap_.get(), result); + } + + void CheckReadResult(const RecordBatch& result, const RecordBatch& expected) { + EXPECT_EQ(expected.num_rows(), result.num_rows()); + + ASSERT_TRUE(expected.schema()->Equals(result.schema())); + ASSERT_EQ(expected.num_columns(), result.num_columns()) + << expected.schema()->ToString() + << " result: " << result.schema()->ToString(); + + for (int i = 0; i < expected.num_columns(); ++i) { + const auto& left = *expected.column(i); + const auto& right = *result.column(i); if (!left.Equals(right)) { std::stringstream pp_result; std::stringstream pp_expected; @@ -178,6 +191,18 @@ class IpcTestFixture : public io::MemoryMapFixture { } } + void CheckRoundtrip(const RecordBatch& batch, int64_t buffer_size) { + std::string path = "test-write-row-batch"; + ASSERT_OK(io::MemoryMapFixture::InitMemoryMap(buffer_size, path, &mmap_)); + + std::shared_ptr result; + ASSERT_OK(DoStandardRoundTrip(batch, &result)); + CheckReadResult(*result, batch); + + // ASSERT_OK(DoLargeRoundTrip(batch, &result)); + // CheckReadResult(*result, batch); + } + void CheckRoundtrip(const std::shared_ptr& array, int64_t buffer_size) { auto f0 = arrow::field("f0", array->type()); std::vector> fields = {f0}; @@ -203,8 +228,6 @@ class TestIpcRoundTrip : public ::testing::TestWithParam, public: void SetUp() { pool_ = default_memory_pool(); } void TearDown() { io::MemoryMapFixture::TearDown(); } - using IpcTestFixture::RoundTripHelper; - using IpcTestFixture::CheckRoundtrip; }; TEST_P(TestIpcRoundTrip, RoundTrip) { diff --git a/cpp/src/arrow/ipc/reader.h b/cpp/src/arrow/ipc/reader.h index c6ec373eead..1c1314a040b 100644 --- a/cpp/src/arrow/ipc/reader.h +++ b/cpp/src/arrow/ipc/reader.h @@ -125,8 +125,8 @@ class ARROW_EXPORT FileReader { /// EXPERIMENTAL: Read length-prefixed LargeRecordBatch metadata (64-bit array /// lengths) at offset and reconstruct RecordBatch -Status ReadLargeRecordBatch(const std::shared_ptr& schema, int64_t offset, - io::RandomAccessFile* file, std::shared_ptr* out); +Status ARROW_EXPORT ReadLargeRecordBatch(const std::shared_ptr& schema, + int64_t offset, io::RandomAccessFile* file, std::shared_ptr* out); } // namespace ipc } // namespace arrow From 36c3862c2c1adfe2b56ea5a40805a282e0f904f4 Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 16:38:38 -0400 Subject: [PATCH 6/9] Get LargeRecordBatch round trip working. Add to Message union for now Change-Id: I652fbd1cf1859b77e5efb96e0cf9a10d8c8cb8d4 --- cpp/src/arrow/ipc/ipc-read-write-test.cc | 8 +++----- cpp/src/arrow/ipc/metadata.cc | 5 ++--- cpp/src/arrow/ipc/reader.cc | 19 ++++++++++++++----- cpp/src/arrow/ipc/test-common.h | 2 +- format/Message.fbs | 6 +++++- 5 files changed, 25 insertions(+), 15 deletions(-) diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc index 2e7d98fa32d..0e63f570b9f 100644 --- a/cpp/src/arrow/ipc/ipc-read-write-test.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc @@ -163,7 +163,6 @@ class IpcTestFixture : public io::MemoryMapFixture { RETURN_NOT_OK(WriteLargeRecordBatch( batch, buffer_offset, mmap_.get(), &metadata_length, &body_length, pool_)); - return ReadLargeRecordBatch(batch.schema(), 0, mmap_.get(), result); } @@ -172,8 +171,7 @@ class IpcTestFixture : public io::MemoryMapFixture { ASSERT_TRUE(expected.schema()->Equals(result.schema())); ASSERT_EQ(expected.num_columns(), result.num_columns()) - << expected.schema()->ToString() - << " result: " << result.schema()->ToString(); + << expected.schema()->ToString() << " result: " << result.schema()->ToString(); for (int i = 0; i < expected.num_columns(); ++i) { const auto& left = *expected.column(i); @@ -199,8 +197,8 @@ class IpcTestFixture : public io::MemoryMapFixture { ASSERT_OK(DoStandardRoundTrip(batch, &result)); CheckReadResult(*result, batch); - // ASSERT_OK(DoLargeRoundTrip(batch, &result)); - // CheckReadResult(*result, batch); + ASSERT_OK(DoLargeRoundTrip(batch, &result)); + CheckReadResult(*result, batch); } void CheckRoundtrip(const std::shared_ptr& array, int64_t buffer_size) { diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc index a3605809424..a418d4893dd 100644 --- a/cpp/src/arrow/ipc/metadata.cc +++ b/cpp/src/arrow/ipc/metadata.cc @@ -629,9 +629,8 @@ Status WriteLargeRecordBatchMessage(int64_t length, int64_t body_length, LargeRecordBatchOffset large_batch; RETURN_NOT_OK( MakeLargeRecordBatch(fbb, length, body_length, nodes, buffers, &large_batch)); - - fbb.Finish(large_batch); - return WriteFlatbufferBuilder(fbb, out); + return WriteMessage(fbb, flatbuf::MessageHeader_LargeRecordBatch, large_batch.Union(), + body_length, out); } Status WriteDictionaryMessage(int64_t id, int32_t length, int64_t body_length, diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc index 606d382c729..a2b20a901a6 100644 --- a/cpp/src/arrow/ipc/reader.cc +++ b/cpp/src/arrow/ipc/reader.cc @@ -512,15 +512,24 @@ class LargeRecordBatchSource : public ArrayComponentSource { Status ReadLargeRecordBatch(const std::shared_ptr& schema, int64_t offset, io::RandomAccessFile* file, std::shared_ptr* out) { std::shared_ptr buffer; - RETURN_NOT_OK(file->ReadAt(offset, sizeof(int32_t), &buffer)); + RETURN_NOT_OK(file->Seek(offset)); + + RETURN_NOT_OK(file->Read(sizeof(int32_t), &buffer)); int32_t flatbuffer_size = *reinterpret_cast(buffer->data()); - RETURN_NOT_OK(file->ReadAt(offset + sizeof(int32_t), flatbuffer_size, &buffer)); + RETURN_NOT_OK(file->Read(flatbuffer_size, &buffer)); + auto message = flatbuf::GetMessage(buffer->data()); + auto batch = reinterpret_cast(message->header()); + + // TODO(ARROW-388): The buffer offsets start at 0, so we must construct a + // RandomAccessFile according to that frame of reference + std::shared_ptr buffer_payload; + RETURN_NOT_OK(file->Read(message->bodyLength(), &buffer_payload)); + io::BufferReader buffer_reader(buffer_payload); - auto metadata = flatbuffers::GetRoot(buffer->data()); - LargeRecordBatchSource source(metadata, file); + LargeRecordBatchSource source(batch, &buffer_reader); return LoadRecordBatchFromSource( - schema, metadata->length(), kMaxNestingDepth, &source, out); + schema, batch->length(), kMaxNestingDepth, &source, out); } } // namespace ipc diff --git a/cpp/src/arrow/ipc/test-common.h b/cpp/src/arrow/ipc/test-common.h index 66a5e09362c..ba203b090b3 100644 --- a/cpp/src/arrow/ipc/test-common.h +++ b/cpp/src/arrow/ipc/test-common.h @@ -103,7 +103,7 @@ Status MakeRandomListArray(const std::shared_ptr& child_array, int num_li typedef Status MakeRecordBatch(std::shared_ptr* out); Status MakeIntRecordBatch(std::shared_ptr* out) { - const int length = 1000; + const int length = 10; // Make the schema auto f0 = field("f0", int32()); diff --git a/format/Message.fbs b/format/Message.fbs index f3379f46134..b632b777f8a 100644 --- a/format/Message.fbs +++ b/format/Message.fbs @@ -335,8 +335,12 @@ table DictionaryBatch { /// This union enables us to easily send different message types without /// redundant storage, and in the future we can easily add new message types. +/// +/// Arrow implementations do not need to implement all of the message types, +/// which may include experimental metadata types. For maximum compatibility, +/// it is best to send data using RecordBatch union MessageHeader { - Schema, DictionaryBatch, RecordBatch + Schema, DictionaryBatch, RecordBatch, LargeRecordBatch } table Message { From 179a1e390d671e5d52938261d128b6bbc6e6680d Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 18:45:38 -0400 Subject: [PATCH 7/9] Add unit test for large record batches. Use bytewise comparisons with aligned bitmaps Change-Id: I92703a6bb6a6d20876efe9fac2844b9b15f4ba12 --- cpp/src/arrow/ipc/ipc-read-write-test.cc | 45 ++++++++++++++++++++---- cpp/src/arrow/util/bit-util.cc | 16 ++++++++- 2 files changed, 53 insertions(+), 8 deletions(-) diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc index 0e63f570b9f..261ca1d0e52 100644 --- a/cpp/src/arrow/ipc/ipc-read-write-test.cc +++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc @@ -125,14 +125,14 @@ TEST_F(TestSchemaMetadata, NestedFields) { class IpcTestFixture : public io::MemoryMapFixture { public: - Status DoStandardRoundTrip( - const RecordBatch& batch, std::shared_ptr* batch_result) { + Status DoStandardRoundTrip(const RecordBatch& batch, bool zero_data, + std::shared_ptr* batch_result) { int32_t metadata_length; int64_t body_length; const int64_t buffer_offset = 0; - 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( @@ -152,13 +152,13 @@ class IpcTestFixture : public io::MemoryMapFixture { } Status DoLargeRoundTrip( - const RecordBatch& batch, std::shared_ptr* result) { + const RecordBatch& batch, bool zero_data, std::shared_ptr* result) { int32_t metadata_length; int64_t body_length; const int64_t buffer_offset = 0; - RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); + if (zero_data) { RETURN_NOT_OK(ZeroMemoryMap(mmap_.get())); } RETURN_NOT_OK(mmap_->Seek(0)); RETURN_NOT_OK(WriteLargeRecordBatch( @@ -194,10 +194,10 @@ class IpcTestFixture : public io::MemoryMapFixture { ASSERT_OK(io::MemoryMapFixture::InitMemoryMap(buffer_size, path, &mmap_)); std::shared_ptr result; - ASSERT_OK(DoStandardRoundTrip(batch, &result)); + ASSERT_OK(DoStandardRoundTrip(batch, true, &result)); CheckReadResult(*result, batch); - ASSERT_OK(DoLargeRoundTrip(batch, &result)); + ASSERT_OK(DoLargeRoundTrip(batch, true, &result)); CheckReadResult(*result, batch); } @@ -538,6 +538,37 @@ INSTANTIATE_TEST_CASE_P(GenericIpcRoundTripTests, TestIpcRoundTrip, BATCH_CASES( INSTANTIATE_TEST_CASE_P(FileRoundTripTests, TestFileFormat, BATCH_CASES()); INSTANTIATE_TEST_CASE_P(StreamRoundTripTests, TestStreamFormat, BATCH_CASES()); +TEST_F(TestIpcRoundTrip, LargeRecordBatch) { + const int64_t length = static_cast(std::numeric_limits::max()) + 1; + + BooleanBuilder builder(default_memory_pool()); + ASSERT_OK(builder.Reserve(length)); + ASSERT_OK(builder.Advance(length)); + + std::shared_ptr array; + ASSERT_OK(builder.Finish(&array)); + + auto f0 = arrow::field("f0", array->type()); + std::vector> fields = {f0}; + auto schema = std::make_shared(fields); + + RecordBatch batch(schema, 0, {array}); + + std::string path = "test-write-large-record_batch"; + + // 512 MB + constexpr int64_t kBufferSize = 1 << 29; + + ASSERT_OK(io::MemoryMapFixture::InitMemoryMap(kBufferSize, path, &mmap_)); + + std::shared_ptr result; + ASSERT_OK(DoLargeRoundTrip(batch, false, &result)); + CheckReadResult(*result, batch); + + // Fails if we try to write this with the normal code path + ASSERT_RAISES(Invalid, DoStandardRoundTrip(batch, false, &result)); +} + void CheckBatchDictionaries(const RecordBatch& batch) { // Check that dictionaries that should be the same are the same auto schema = batch.schema(); diff --git a/cpp/src/arrow/util/bit-util.cc b/cpp/src/arrow/util/bit-util.cc index 3767ba9e62f..ba0bfd7a9e3 100644 --- a/cpp/src/arrow/util/bit-util.cc +++ b/cpp/src/arrow/util/bit-util.cc @@ -112,7 +112,21 @@ Status CopyBitmap(MemoryPool* pool, const uint8_t* data, int64_t offset, int64_t bool BitmapEquals(const uint8_t* left, int64_t left_offset, const uint8_t* right, int64_t right_offset, int64_t bit_length) { - // TODO(wesm): Make this faster using word-wise comparisons + if (left_offset % 8 == 0 && right_offset % 8 == 0) { + // byte aligned, can use memcmp + bool bytes_equal = std::memcmp(left + left_offset / 8, right + right_offset / 8, + bit_length / 8) == 0; + if (!bytes_equal) { return false; } + for (int64_t i = (bit_length / 8) * 8; i < bit_length; ++i) { + if (BitUtil::GetBit(left, left_offset + i) != + BitUtil::GetBit(right, right_offset + i)) { + return false; + } + } + return true; + } + + // Unaligned slow case for (int64_t i = 0; i < bit_length; ++i) { if (BitUtil::GetBit(left, left_offset + i) != BitUtil::GetBit(right, right_offset + i)) { From d7811f2d60f8b807ab7380662f630397da7cbb7e Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 18:49:28 -0400 Subject: [PATCH 8/9] cpplint Change-Id: Ifd0181a256aabb24625d94cd01b5fb8e681e9ba7 --- cpp/src/arrow/ipc/writer.cc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc index 0c0d693a2cd..82c119ef53e 100644 --- a/cpp/src/arrow/ipc/writer.cc +++ b/cpp/src/arrow/ipc/writer.cc @@ -17,8 +17,10 @@ #include "arrow/ipc/writer.h" +#include #include #include +#include #include #include From 9c18a95e309f4bfe68915c7b9ccf51965e9f6193 Mon Sep 17 00:00:00 2001 From: Wes McKinney Date: Sun, 19 Mar 2017 19:03:45 -0400 Subject: [PATCH 9/9] Fix import ordering Change-Id: I9fe7248278965688c96a48917191d1e2c7b0fb9f --- cpp/src/arrow/allocator-test.cc | 3 ++- cpp/src/arrow/allocator.h | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/cpp/src/arrow/allocator-test.cc b/cpp/src/arrow/allocator-test.cc index a78d170ba12..811ef5a79c2 100644 --- a/cpp/src/arrow/allocator-test.cc +++ b/cpp/src/arrow/allocator-test.cc @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. +#include "gtest/gtest.h" + #include "arrow/allocator.h" #include "arrow/test-util.h" -#include "gtest/gtest.h" namespace arrow { diff --git a/cpp/src/arrow/allocator.h b/cpp/src/arrow/allocator.h index 6a054aea228..e00023dc460 100644 --- a/cpp/src/arrow/allocator.h +++ b/cpp/src/arrow/allocator.h @@ -18,12 +18,13 @@ #ifndef ARROW_ALLOCATOR_H #define ARROW_ALLOCATOR_H -#include "arrow/memory_pool.h" -#include "arrow/status.h" #include #include #include +#include "arrow/memory_pool.h" +#include "arrow/status.h" + namespace arrow { template