You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/07/06 14:12:45 UTC

[GitHub] [arrow] lidavidm opened a new pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

lidavidm opened a new pull request #10663:
URL: https://github.com/apache/arrow/pull/10663


   We can't report errors during serialization - gRPC will just trip an assert. Instead, move these checks into the layer above so we can report them to the client or server as appropriate.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-879533552


   Benchmark runs are scheduled for baseline = 780e95c512d63bbea1e040af0eb44a0bf63c4d72 and contender = 64ecb2a73c76e30091428af5e03d538766b12c74. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/399dd7ae47d345969dfba70ba5d60fae...720759ec32734f10a4016acec5cc4633/)
   [Scheduled] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/eef93c4bfb1d4a8b92351f41883dfd55...41a8348d634e4410808dad7fcdd62c59/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/e5af519d38bd4cc483cd45f86750d807...b477c89467fc4b2d80211db405bd786e/)
   Supported benchmarks:
   ursa-i9-9960x: langs = Python, R
   ursa-thinkcentre-m75q: langs = C++, Java
   ec2-t3-xlarge-us-east-2: cloud = True
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] dianaclarke commented on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
dianaclarke commented on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-879533345


   @ursabot please benchmark


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r669633930



##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));

Review comment:
       I'll try to get a repro up when I can and file a bug, good idea. 

##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));
-    }
+    DCHECK_LT(msg.descriptor->size(), kInt32Max);

Review comment:
       Whoops, fixed (here and elsewhere), thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r669705163



##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));

Review comment:
       See https://github.com/grpc/grpc/issues/26695




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] github-actions[bot] commented on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-874796945


   https://issues.apache.org/jira/browse/ARROW-13253


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-879533552


   Benchmark runs are scheduled for baseline = 780e95c512d63bbea1e040af0eb44a0bf63c4d72 and contender = 64ecb2a73c76e30091428af5e03d538766b12c74. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/399dd7ae47d345969dfba70ba5d60fae...720759ec32734f10a4016acec5cc4633/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/eef93c4bfb1d4a8b92351f41883dfd55...41a8348d634e4410808dad7fcdd62c59/)
   [Finished :arrow_down:0.24% :arrow_up:0.05%] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/e5af519d38bd4cc483cd45f86750d807...b477c89467fc4b2d80211db405bd786e/)
   Supported benchmarks:
   ursa-i9-9960x: langs = Python, R
   ursa-thinkcentre-m75q: langs = C++, Java
   ec2-t3-xlarge-us-east-2: cloud = True
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot commented on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-879533552


   Benchmark runs are scheduled for baseline = 780e95c512d63bbea1e040af0eb44a0bf63c4d72 and contender = 64ecb2a73c76e30091428af5e03d538766b12c74. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/399dd7ae47d345969dfba70ba5d60fae...720759ec32734f10a4016acec5cc4633/)
   [Scheduled] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/eef93c4bfb1d4a8b92351f41883dfd55...41a8348d634e4410808dad7fcdd62c59/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/e5af519d38bd4cc483cd45f86750d807...b477c89467fc4b2d80211db405bd786e/)
   Supported benchmarks:
   ursa-i9-9960x: langs = Python, R
   ursa-thinkcentre-m75q: langs = C++, Java
   ec2-t3-xlarge-us-east-2: cloud = True
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] cyb70289 closed pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
cyb70289 closed pull request #10663:
URL: https://github.com/apache/arrow/pull/10663


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r669634201



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -688,11 +688,12 @@ class GrpcStreamWriter : public FlightStreamWriter {
   Status WriteMetadata(std::shared_ptr<Buffer> app_metadata) override {
     FlightPayload payload{};
     payload.app_metadata = app_metadata;
-    if (!internal::WritePayload(payload, writer_->stream().get())) {
+    auto status = internal::WritePayload(payload, writer_->stream().get());
+    if (!status.ok() && status.IsIOError()) {

Review comment:
       Good point, it's redundant. Fixed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-880690668


   Thanks for the review! :slightly_smiling_face:


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] cyb70289 commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r669247552



##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));
-    }
+    DCHECK_LT(msg.descriptor->size(), kInt32Max);

Review comment:
       It's a bit strange to continue running here, if we know it will fail finally (will it?)
   
   From the jira issue, looks grpc should check and handle our return value decently.
   Maybe open an issue in grpc community? If grpc is willing to change, do we still need to update our code?
   
   For now, I'm not sure if it's appropriate to simply abort here in release build, with some clear error messages and hints to fix it by chunking the data.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] cyb70289 commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r669232404



##########
File path: cpp/src/arrow/flight/client.cc
##########
@@ -688,11 +688,12 @@ class GrpcStreamWriter : public FlightStreamWriter {
   Status WriteMetadata(std::shared_ptr<Buffer> app_metadata) override {
     FlightPayload payload{};
     payload.app_metadata = app_metadata;
-    if (!internal::WritePayload(payload, writer_->stream().get())) {
+    auto status = internal::WritePayload(payload, writer_->stream().get());
+    if (!status.ok() && status.IsIOError()) {

Review comment:
       Can we remove `!status.ok()` and only check `if (status.IsIOError())` ?

##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));
-    }
+    DCHECK_LT(msg.descriptor->size(), kInt32Max);

Review comment:
       `DCHECK_LE` ?

##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));
-    }
+    DCHECK_LT(msg.descriptor->size(), kInt32Max);

Review comment:
       It's a bit strange to continue running here, if we know it will fail finally (will it?)
   
   From the jira issue, looks grpc should check and handle our return value decently.
   Maybe open an issue in grpc community? If grpc is willing to change, do we still need to update our code?
   
   For now, I'm not sure if it's appropriate to simply abort here in release build, with some clear error messages and hints to fix it by chunking the data.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r670405214



##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -164,26 +164,18 @@ static const uint8_t kPaddingBytes[8] = {0, 0, 0, 0, 0, 0, 0, 0};
 
 // Update the sizes of our Protobuf fields based on the given IPC payload.
 grpc::Status IpcMessageHeaderSize(const arrow::ipc::IpcPayload& ipc_msg, bool has_body,
-                                  size_t* body_size, size_t* header_size,
-                                  int32_t* metadata_size) {
-  DCHECK_LT(ipc_msg.metadata->size(), kInt32Max);
+                                  size_t* header_size, int32_t* metadata_size) {
+  DCHECK_LE(ipc_msg.metadata->size(), kInt32Max);
   *metadata_size = static_cast<int32_t>(ipc_msg.metadata->size());
 
   // 1 byte for metadata tag
   *header_size += 1 + WireFormatLite::LengthDelimitedSize(*metadata_size);
 
-  for (const auto& buffer : ipc_msg.body_buffers) {
-    // Buffer may be null when the row length is zero, or when all
-    // entries are invalid.
-    if (!buffer) continue;
-
-    *body_size += static_cast<size_t>(BitUtil::RoundUpToMultipleOf8(buffer->size()));
-  }
-
   // 2 bytes for body tag
   if (has_body) {
     // We write the body tag in the header but not the actual body data
-    *header_size += 2 + WireFormatLite::LengthDelimitedSize(*body_size) - *body_size;
+    *header_size += 2 + WireFormatLite::LengthDelimitedSize(ipc_msg.body_length) -

Review comment:
       If you constructed your own IPC payload, I suppose that could be the case. But in general you'd use one of the GetFooPayload functions in arrow/ipc/writer.h, which precalculate the body length for you. (And it is valid to have a zero-length payload in general, since it might be metadata-only or IPC-header-only. The previous calculation here was solely to guard against a too-large payload, but the only thing we can do at this point is crash anyways.)

##########
File path: cpp/src/arrow/flight/types.cc
##########
@@ -21,6 +21,7 @@
 #include <sstream>
 #include <utility>
 
+#include "arrow/buffer.h"

Review comment:
       It's actually needed for the descriptor->size() call below.

##########
File path: cpp/src/arrow/flight/test_util.cc
##########
@@ -616,6 +640,22 @@ Status ExampleLargeBatches(BatchVector* out) {
   return Status::OK();
 }
 
+arrow::Result<std::shared_ptr<RecordBatch>> VeryLargeBatch() {
+  // In CI, some platforms don't let us allocate one very large
+  // buffer, so allocate a smaller buffer and repeat it a few times
+  constexpr int64_t nbytes = (1ul << 27ul) + 8ul;
+  constexpr int64_t nrows = nbytes / 8;
+  constexpr int64_t ncols = 16;
+  ARROW_ASSIGN_OR_RAISE(auto values, AllocateBuffer(nbytes));
+  std::memset(values->mutable_data(), 0x00, values->capacity());
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, std::move(values)};
+  auto array = std::make_shared<ArrayData>(int64(), nrows, buffers,
+                                           /*null_count=*/0);
+  std::vector<std::shared_ptr<ArrayData>> arrays(ncols, array);
+  std::vector<std::shared_ptr<Field>> fields(ncols, field("a", int64()));
+  return RecordBatch::Make(schema(std::move(fields)), nrows, std::move(arrays));

Review comment:
       Yes, and when written out, we'll write all 16 columns to the stream. (This was done just to save memory and try to get the test runnable on most of the CI environments.)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] cyb70289 commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r670100132



##########
File path: cpp/src/arrow/flight/test_util.cc
##########
@@ -616,6 +640,22 @@ Status ExampleLargeBatches(BatchVector* out) {
   return Status::OK();
 }
 
+arrow::Result<std::shared_ptr<RecordBatch>> VeryLargeBatch() {
+  // In CI, some platforms don't let us allocate one very large
+  // buffer, so allocate a smaller buffer and repeat it a few times
+  constexpr int64_t nbytes = (1ul << 27ul) + 8ul;
+  constexpr int64_t nrows = nbytes / 8;
+  constexpr int64_t ncols = 16;
+  ARROW_ASSIGN_OR_RAISE(auto values, AllocateBuffer(nbytes));
+  std::memset(values->mutable_data(), 0x00, values->capacity());
+  std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, std::move(values)};
+  auto array = std::make_shared<ArrayData>(int64(), nrows, buffers,
+                                           /*null_count=*/0);
+  std::vector<std::shared_ptr<ArrayData>> arrays(ncols, array);
+  std::vector<std::shared_ptr<Field>> fields(ncols, field("a", int64()));
+  return RecordBatch::Make(schema(std::move(fields)), nrows, std::move(arrays));

Review comment:
       So the ipc payload will be 16 columns in size, though all columns are from same buffer?

##########
File path: cpp/src/arrow/flight/types.cc
##########
@@ -21,6 +21,7 @@
 #include <sstream>
 #include <utility>
 
+#include "arrow/buffer.h"

Review comment:
       Is this header used?

##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -164,26 +164,18 @@ static const uint8_t kPaddingBytes[8] = {0, 0, 0, 0, 0, 0, 0, 0};
 
 // Update the sizes of our Protobuf fields based on the given IPC payload.
 grpc::Status IpcMessageHeaderSize(const arrow::ipc::IpcPayload& ipc_msg, bool has_body,
-                                  size_t* body_size, size_t* header_size,
-                                  int32_t* metadata_size) {
-  DCHECK_LT(ipc_msg.metadata->size(), kInt32Max);
+                                  size_t* header_size, int32_t* metadata_size) {
+  DCHECK_LE(ipc_msg.metadata->size(), kInt32Max);
   *metadata_size = static_cast<int32_t>(ipc_msg.metadata->size());
 
   // 1 byte for metadata tag
   *header_size += 1 + WireFormatLite::LengthDelimitedSize(*metadata_size);
 
-  for (const auto& buffer : ipc_msg.body_buffers) {
-    // Buffer may be null when the row length is zero, or when all
-    // entries are invalid.
-    if (!buffer) continue;
-
-    *body_size += static_cast<size_t>(BitUtil::RoundUpToMultipleOf8(buffer->size()));
-  }
-
   // 2 bytes for body tag
   if (has_body) {
     // We write the body tag in the header but not the actual body data
-    *header_size += 2 + WireFormatLite::LengthDelimitedSize(*body_size) - *body_size;
+    *header_size += 2 + WireFormatLite::LengthDelimitedSize(ipc_msg.body_length) -

Review comment:
       From old code, looks `ipc_msg.body_length` may be 0. It's not possible now?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] ursabot edited a comment on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-879533552


   Benchmark runs are scheduled for baseline = 780e95c512d63bbea1e040af0eb44a0bf63c4d72 and contender = 64ecb2a73c76e30091428af5e03d538766b12c74. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/399dd7ae47d345969dfba70ba5d60fae...720759ec32734f10a4016acec5cc4633/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/eef93c4bfb1d4a8b92351f41883dfd55...41a8348d634e4410808dad7fcdd62c59/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/e5af519d38bd4cc483cd45f86750d807...b477c89467fc4b2d80211db405bd786e/)
   Supported benchmarks:
   ursa-i9-9960x: langs = Python, R
   ursa-thinkcentre-m75q: langs = C++, Java
   ec2-t3-xlarge-us-east-2: cloud = True
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] cyb70289 commented on a change in pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on a change in pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#discussion_r669261083



##########
File path: cpp/src/arrow/flight/serialization_internal.cc
##########
@@ -201,9 +193,7 @@ grpc::Status FlightDataSerialize(const FlightPayload& msg, ByteBuffer* out,
   // Write the descriptor if present
   int32_t descriptor_size = 0;
   if (msg.descriptor != nullptr) {
-    if (msg.descriptor->size() > kInt32Max) {
-      return ToGrpcStatus(Status::CapacityError("Descriptor size overflow (>= 2**31)"));

Review comment:
       From the jira issue, looks grpc should check and handle our return value decently.
   Maybe open an issue in grpc community?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on pull request #10663: ARROW-13253: [FlightRPC][C++] Fix segfault with large messages

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #10663:
URL: https://github.com/apache/arrow/pull/10663#issuecomment-879371544


   @cyb70289, might you have time to look at this? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org