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

[arrow] branch master updated: ARROW-5605: [C++] Verify Flatbuffer messages in more places to prevent crashes due to bad inputs

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new b04fae1  ARROW-5605: [C++] Verify Flatbuffer messages in more places to prevent crashes due to bad inputs
b04fae1 is described below

commit b04fae1cb65e56637cb28c6229c65a356e0de147
Author: Marco Neumann <ma...@crepererum.net>
AuthorDate: Fri Jun 21 16:59:06 2019 -0500

    ARROW-5605: [C++] Verify Flatbuffer messages in more places to prevent crashes due to bad inputs
    
    While the first commit (`fix ReadRecordBatch validation`) is sufficient to fix ARROW-5605, I took the time to fix very similar issues in the code IPC code base, so our users are probably protected and also to help the fuzzer to not run straight into a similar problem again.
    
    Author: Marco Neumann <ma...@crepererum.net>
    Author: Marco Neumann <ma...@blue-yonder.com>
    Author: Wes McKinney <we...@apache.org>
    
    Closes #4573 from crepererum/ARROW-5605 and squashes the following commits:
    
    75a3b6d2f <Wes McKinney> Refactor message verification into helper function
    6b528d48b <Marco Neumann> fix linting
    80f6ce2a4 <Marco Neumann> remove risky reinterpret_casts
    7ebdd7559 <Marco Neumann> validate footer
    fb70669b1 <Marco Neumann> remove obsolete reinterpret_cast
    cdb79849c <Marco Neumann> guard all getMessage calls with validation
    27d7dad46 <Marco Neumann> fix ReadRecordBatch validation
---
 cpp/src/arrow/ipc/message.cc           | 20 +++++++---------
 cpp/src/arrow/ipc/metadata-internal.cc | 22 +++++++++--------
 cpp/src/arrow/ipc/metadata-internal.h  | 11 +++++++++
 cpp/src/arrow/ipc/reader.cc            | 43 +++++++++++++++++++++-------------
 4 files changed, 58 insertions(+), 38 deletions(-)

diff --git a/cpp/src/arrow/ipc/message.cc b/cpp/src/arrow/ipc/message.cc
index f6d43e1..1de2a26 100644
--- a/cpp/src/arrow/ipc/message.cc
+++ b/cpp/src/arrow/ipc/message.cc
@@ -43,7 +43,8 @@ class Message::MessageImpl {
       : metadata_(metadata), message_(nullptr), body_(body) {}
 
   Status Open() {
-    message_ = flatbuf::GetMessage(metadata_->data());
+    RETURN_NOT_OK(
+        internal::VerifyMessage(metadata_->data(), metadata_->size(), &message_));
 
     // Check that the metadata version is supported
     if (message_->version() < internal::kMinMetadataVersion) {
@@ -143,12 +144,8 @@ bool Message::Equals(const Message& other) const {
 
 Status Message::ReadFrom(const std::shared_ptr<Buffer>& metadata, io::InputStream* stream,
                          std::unique_ptr<Message>* out) {
-  auto data = metadata->data();
-  flatbuffers::Verifier verifier(data, metadata->size(), /*max_depth=*/128);
-  if (!flatbuf::VerifyMessageBuffer(verifier)) {
-    return Status::IOError("Invalid flatbuffers message.");
-  }
-  auto fb_message = flatbuf::GetMessage(data);
+  const flatbuf::Message* fb_message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata->data(), metadata->size(), &fb_message));
 
   int64_t body_length = fb_message->bodyLength();
 
@@ -164,8 +161,8 @@ Status Message::ReadFrom(const std::shared_ptr<Buffer>& metadata, io::InputStrea
 
 Status Message::ReadFrom(const int64_t offset, const std::shared_ptr<Buffer>& metadata,
                          io::RandomAccessFile* file, std::unique_ptr<Message>* out) {
-  auto fb_message = flatbuf::GetMessage(metadata->data());
-
+  const flatbuf::Message* fb_message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata->data(), metadata->size(), &fb_message));
   int64_t body_length = fb_message->bodyLength();
 
   std::shared_ptr<Buffer> body;
@@ -209,9 +206,8 @@ Status Message::SerializeTo(io::OutputStream* stream, int32_t alignment,
 }
 
 bool Message::Verify() const {
-  std::shared_ptr<Buffer> meta = this->metadata();
-  flatbuffers::Verifier verifier(meta->data(), meta->size(), 128);
-  return flatbuf::VerifyMessageBuffer(verifier);
+  const flatbuf::Message* unused;
+  return internal::VerifyMessage(metadata()->data(), metadata()->size(), &unused).ok();
 }
 
 std::string FormatMessageType(Message::Type type) {
diff --git a/cpp/src/arrow/ipc/metadata-internal.cc b/cpp/src/arrow/ipc/metadata-internal.cc
index 46f3366..4b349cb 100644
--- a/cpp/src/arrow/ipc/metadata-internal.cc
+++ b/cpp/src/arrow/ipc/metadata-internal.cc
@@ -1112,8 +1112,12 @@ Status GetSchema(const void* opaque_schema, DictionaryMemo* dictionary_memo,
 Status GetTensorMetadata(const Buffer& metadata, std::shared_ptr<DataType>* type,
                          std::vector<int64_t>* shape, std::vector<int64_t>* strides,
                          std::vector<std::string>* dim_names) {
-  auto message = flatbuf::GetMessage(metadata.data());
-  auto tensor = reinterpret_cast<const flatbuf::Tensor*>(message->header());
+  const flatbuf::Message* message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message));
+  auto tensor = message->header_as_Tensor();
+  if (tensor == nullptr) {
+    return Status::IOError("Header-type of flatbuffer-encoded Message is not Tensor.");
+  }
 
   int ndim = static_cast<int>(tensor->shape()->size());
 
@@ -1143,15 +1147,13 @@ Status GetSparseTensorMetadata(const Buffer& metadata, std::shared_ptr<DataType>
                                std::vector<std::string>* dim_names,
                                int64_t* non_zero_length,
                                SparseTensorFormat::type* sparse_tensor_format_id) {
-  auto message = flatbuf::GetMessage(metadata.data());
-  if (message->header_type() != flatbuf::MessageHeader_SparseTensor) {
-    return Status::IOError("Header of flatbuffer-encoded Message is not SparseTensor.");
-  }
-  if (message->header() == nullptr) {
-    return Status::IOError("Header-pointer of flatbuffer-encoded Message is null.");
+  const flatbuf::Message* message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message));
+  auto sparse_tensor = message->header_as_SparseTensor();
+  if (sparse_tensor == nullptr) {
+    return Status::IOError(
+        "Header-type of flatbuffer-encoded Message is not SparseTensor.");
   }
-
-  auto sparse_tensor = reinterpret_cast<const flatbuf::SparseTensor*>(message->header());
   int ndim = static_cast<int>(sparse_tensor->shape()->size());
 
   for (int i = 0; i < ndim; ++i) {
diff --git a/cpp/src/arrow/ipc/metadata-internal.h b/cpp/src/arrow/ipc/metadata-internal.h
index 4563fb0..94adf64 100644
--- a/cpp/src/arrow/ipc/metadata-internal.h
+++ b/cpp/src/arrow/ipc/metadata-internal.h
@@ -29,6 +29,7 @@
 #include <flatbuffers/flatbuffers.h>
 
 #include "arrow/buffer.h"
+#include "arrow/ipc/Message_generated.h"
 #include "arrow/ipc/Schema_generated.h"
 #include "arrow/ipc/dictionary.h"  // IYWU pragma: keep
 #include "arrow/ipc/message.h"
@@ -108,6 +109,16 @@ Status GetSparseTensorMetadata(const Buffer& metadata, std::shared_ptr<DataType>
                                std::vector<std::string>* dim_names, int64_t* length,
                                SparseTensorFormat::type* sparse_tensor_format_id);
 
+static inline Status VerifyMessage(const uint8_t* data, int64_t size,
+                                   const flatbuf::Message** out) {
+  flatbuffers::Verifier verifier(data, size, /*max_depth=*/128);
+  if (!flatbuf::VerifyMessageBuffer(verifier)) {
+    return Status::IOError("Invalid flatbuffers message.");
+  }
+  *out = flatbuf::GetMessage(data);
+  return Status::OK();
+}
+
 /// Write a serialized message metadata with a length-prefix and padding to an
 /// 8-byte offset. Does not make assumptions about whether the stream is
 /// aligned already
diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc
index c870ca6..717f29a 100644
--- a/cpp/src/arrow/ipc/reader.cc
+++ b/cpp/src/arrow/ipc/reader.cc
@@ -378,22 +378,25 @@ static inline Status ReadRecordBatch(const flatbuf::RecordBatch* metadata,
 Status ReadRecordBatch(const Buffer& metadata, const std::shared_ptr<Schema>& schema,
                        const DictionaryMemo* dictionary_memo, int max_recursion_depth,
                        io::RandomAccessFile* file, std::shared_ptr<RecordBatch>* out) {
-  auto message = flatbuf::GetMessage(metadata.data());
-  if (message->header_type() != flatbuf::MessageHeader_RecordBatch) {
-    DCHECK_EQ(message->header_type(), flatbuf::MessageHeader_RecordBatch);
+  const flatbuf::Message* message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message));
+  auto batch = message->header_as_RecordBatch();
+  if (batch == nullptr) {
+    return Status::IOError(
+        "Header-type of flatbuffer-encoded Message is not RecordBatch.");
   }
-  if (message->header() == nullptr) {
-    return Status::IOError("Header-pointer of flatbuffer-encoded Message is null.");
-  }
-  auto batch = reinterpret_cast<const flatbuf::RecordBatch*>(message->header());
   return ReadRecordBatch(batch, schema, dictionary_memo, max_recursion_depth, file, out);
 }
 
 Status ReadDictionary(const Buffer& metadata, DictionaryMemo* dictionary_memo,
                       io::RandomAccessFile* file) {
-  auto message = flatbuf::GetMessage(metadata.data());
-  auto dictionary_batch =
-      reinterpret_cast<const flatbuf::DictionaryBatch*>(message->header());
+  const flatbuf::Message* message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message));
+  auto dictionary_batch = message->header_as_DictionaryBatch();
+  if (dictionary_batch == nullptr) {
+    return Status::IOError(
+        "Header-type of flatbuffer-encoded Message is not DictionaryBatch.");
+  }
 
   int64_t id = dictionary_batch->id();
 
@@ -406,8 +409,7 @@ Status ReadDictionary(const Buffer& metadata, DictionaryMemo* dictionary_memo,
 
   // The dictionary is embedded in a record batch with a single column
   std::shared_ptr<RecordBatch> batch;
-  auto batch_meta =
-      reinterpret_cast<const flatbuf::RecordBatch*>(dictionary_batch->data());
+  auto batch_meta = dictionary_batch->data();
   RETURN_NOT_OK(ReadRecordBatch(batch_meta, ::arrow::schema({value_field}),
                                 dictionary_memo, kMaxNestingDepth, file, &batch));
   if (batch->num_columns() != 1) {
@@ -606,8 +608,12 @@ class RecordBatchFileReader::RecordBatchFileReaderImpl {
     RETURN_NOT_OK(file_->ReadAt(footer_offset_ - footer_length - file_end_size,
                                 footer_length, &footer_buffer_));
 
-    // TODO(wesm): Verify the footer
-    footer_ = flatbuf::GetFooter(footer_buffer_->data());
+    auto data = footer_buffer_->data();
+    flatbuffers::Verifier verifier(data, footer_buffer_->size(), 128);
+    if (!flatbuf::VerifyFooterBuffer(verifier)) {
+      return Status::IOError("Verification of flatbuffer-encoded Footer failed.");
+    }
+    footer_ = flatbuf::GetFooter(data);
 
     return Status::OK();
   }
@@ -879,8 +885,13 @@ Status ReadSparseTensor(const Buffer& metadata, io::RandomAccessFile* file,
   RETURN_NOT_OK(internal::GetSparseTensorMetadata(
       metadata, &type, &shape, &dim_names, &non_zero_length, &sparse_tensor_format_id));
 
-  auto message = flatbuf::GetMessage(metadata.data());
-  auto sparse_tensor = reinterpret_cast<const flatbuf::SparseTensor*>(message->header());
+  const flatbuf::Message* message;
+  RETURN_NOT_OK(internal::VerifyMessage(metadata.data(), metadata.size(), &message));
+  auto sparse_tensor = message->header_as_SparseTensor();
+  if (sparse_tensor == nullptr) {
+    return Status::IOError(
+        "Header-type of flatbuffer-encoded Message is not SparseTensor.");
+  }
   const flatbuf::Buffer* buffer = sparse_tensor->data();
   DCHECK(BitUtil::IsMultipleOf8(buffer->offset()))
       << "Buffer of sparse index data "