@@ -2,7 +2,7 @@ use crate::common::ttl_map::{TTLMap, TTLMapConfig};
22use crate :: flight_service:: DistributedSessionBuilder ;
33use crate :: flight_service:: do_get:: TaskData ;
44use crate :: protobuf:: StageKey ;
5- use arrow_flight:: flight_service_server:: FlightService ;
5+ use arrow_flight:: flight_service_server:: { FlightService , FlightServiceServer } ;
66use arrow_flight:: {
77 Action , ActionType , Criteria , Empty , FlightData , FlightDescriptor , FlightInfo ,
88 HandshakeRequest , HandshakeResponse , PollInfo , PutResult , SchemaResult , Ticket ,
@@ -41,7 +41,7 @@ impl ArrowFlightEndpoint {
4141 task_data_entries : Arc :: new ( ttl_map) ,
4242 session_builder : Arc :: new ( session_builder) ,
4343 hooks : ArrowFlightEndpointHooks :: default ( ) ,
44- max_message_size : None ,
44+ max_message_size : Some ( usize :: MAX ) ,
4545 } )
4646 }
4747
@@ -59,20 +59,41 @@ impl ArrowFlightEndpoint {
5959
6060 /// Set the maximum message size for FlightData chunks.
6161 ///
62- /// Defaults to None, which uses `arrow-rs` default, curerntly 2MB .
62+ /// Defaults to `usize::MAX` to minimize chunking overhead for internal communication .
6363 /// See [`FlightDataEncoderBuilder::with_max_flight_data_size`] for details.
6464 ///
65- /// If you change this, ensure you configure the server's max_encoding_message_size and
66- /// max_decoding_message_size to at least 2x this value to allow for overhead.
67- /// If your service communication is purely internal and there is no risk of DOS attacks,
68- /// you may want to set this to a considerably larger value to minimize the overhead of chunking
69- /// larger datasets.
65+ /// If you change this to a lower value, ensure you configure the server's
66+ /// max_encoding_message_size and max_decoding_message_size to at least 2x this value
67+ /// to allow for overhead. For most use cases, the default of `usize::MAX` is appropriate.
7068 ///
7169 /// [`FlightDataEncoderBuilder::with_max_flight_data_size`]: https://arrow.apache.org/rust/arrow_flight/encode/struct.FlightDataEncoderBuilder.html#structfield.max_flight_data_size
7270 pub fn with_max_message_size ( mut self , size : usize ) -> Self {
7371 self . max_message_size = Some ( size) ;
7472 self
7573 }
74+
75+ /// Converts this endpoint into a [`FlightServiceServer`] with high default message size limits.
76+ ///
77+ /// This is a convenience method that wraps the endpoint in a [`FlightServiceServer`] and
78+ /// configures it with `max_decoding_message_size(usize::MAX)` and
79+ /// `max_encoding_message_size(usize::MAX)` to avoid message size limitations for internal
80+ /// communication.
81+ ///
82+ /// You can further customize the returned server by chaining additional tonic methods.
83+ ///
84+ /// # Example
85+ ///
86+ /// ```rust,ignore
87+ /// let endpoint = ArrowFlightEndpoint::try_new(session_builder)?;
88+ /// let server = endpoint.into_flight_server();
89+ /// // Can chain additional tonic methods if needed
90+ /// // let server = server.some_other_tonic_method(...);
91+ /// ```
92+ pub fn into_flight_server ( self ) -> FlightServiceServer < Self > {
93+ FlightServiceServer :: new ( self )
94+ . max_decoding_message_size ( usize:: MAX )
95+ . max_encoding_message_size ( usize:: MAX )
96+ }
7697}
7798
7899#[ async_trait]
0 commit comments