You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "langtsai (via GitHub)" <gi...@apache.org> on 2023/03/01 14:16:14 UTC

[GitHub] [arrow] langtsai opened a new issue, #34402: [C++][Flight] Unable to receive error status from DoPut

langtsai opened a new issue, #34402:
URL: https://github.com/apache/arrow/issues/34402

   ### Describe the bug, including details regarding any error messages, version, and platform.
   
   Hi I am am running into an issue where my Flight server DoPut is returning an error, but I am unable to receive it on the client side.  Eventually after attempting to write data an exception is triggered on closing of the writer.  And that exception was the error that should have been triggered at the time of calling DoPut.  For my setup I am relying on the initial call (to client->DoPut) to authenticate and validate whether the client has write access. If successful the client would then proceed to write data via the writer.  However in this scenario, my server has rejected the (DoPut) request and the client has no knowledge of this and attempts to proceed to write anyway.  Does anyone know how I can retrieve the DoPut error right after the invocation? Below is a snippet of my issue
   
   ```
   // Server side code that just returns an error
   arrow::Status FlightServer::DoPut(const arrow::flight::ServerCallContext& context,
                                     std::unique_ptr<arrow::flight::FlightMessageReader> reader,
                                     std::unique_ptr<arrow::flight::FlightMetadataWriter> writer)
   {
       return flight::MakeFlightError(flight::FlightStatusCode::Failed, "Testing DoPut failure");
   }
   ```
   
   ```
   // Client side code
   int client_main(int argc, char* argv[])
   {
       try {
           run_do_put_test();
       } 
       catch (const std::exception& e) {
         std::cout << "Error running test: " << e.what() << std::endl;
         return EXIT_FAILURE;
       }
   
       return EXIT_SUCCESS;
   }
   
   void run_do_put_test()
   {
       arrow::Status status = run_do_put_impl();
       if (!status.ok())
         throw std::runtime_error(status.message());
   }
   
   arrow::Status run_do_put_impl()
   {
       std::unique_ptr<flight::FlightClient> client;
       ARROW_ASSIGN_OR_RAISE(flight::Location connect_location, flight::Location::ForGrpcTcp("0.0.0.0", 8001));
       ARROW_ASSIGN_OR_RAISE(client, flight::FlightClient::Connect(connect_location));
   
       auto do_put_input = get_input();
       flight::FlightCallOptions call_options;
       const auto fd = arrow::flight::FlightDescriptor::Command(do_put_input);
   
       // The call to DoPut is expected to result in failure, but it doesn not...
       ARROW_ASSIGN_OR_RAISE(auto result, client->DoPut(call_options, fd, get_schema()));
       
       auto writer = std::move(result.writer);
       auto metadata_reader = std::move(result.reader);
       auto metadata = get_metadata();
       const std::shared_ptr<arrow::Table> table = get_write_table();
       arrow::TableBatchReader batch_reader(table);
       while(true) {
         std::shared_ptr<arrow::RecordBatch> batch;
         ARROW_RETURN_NOT_OK(batch_reader.ReadNext(&batch));
   
         if(!batch) {
           break;
         }
         std::cout << "Attempting to write metadata" << std::endl;
         ARROW_RETURN_NOT_OK(writer->WriteWithMetadata(*batch, metadata));
       }
   
       ARROW_RETURN_NOT_OK(writer->DoneWriting());
       std::cout << "Done Writing" << std::endl;
   
       // This next line is where the exception for DoPut will get triggered...
       ARROW_RETURN_NOT_OK(writer->Close()); 
       std::cout << "Closed Writer" << std::endl;
   
       while(true) {
         std::shared_ptr<arrow::Buffer> out_metadata;
         ARROW_RETURN_NOT_OK(metadata_reader->ReadMetadata(&out_metadata));
   
         if(!out_metadata) {
           break;
         }
   
         std::cout << out_metadata->ToString() << std::endl;
       }
   
       return arrow::Status::OK();
   }
   ```
   
   Output from the program execution:
   
   > Attempting to write metadata
   > Done Writing
   > Error running test: Testing DoPut failure. Detail: Failed. gRPC client debug context: {"created":"@1677625909.702169698","description":"Error received from peer ipv4:0.0.0.0:8001","file":"/src/core/lib/surface/call.cc","file_line":1041,"grpc_message":"Testing DoPut failure. Detail: Failed","grpc_status":2}. Client context: OK
   
   
   ### Component(s)
   
   C++


-- 
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: issues-unsubscribe@arrow.apache.org.apache.org

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