File tree Expand file tree Collapse file tree 2 files changed +32
-20
lines changed Expand file tree Collapse file tree 2 files changed +32
-20
lines changed Original file line number Diff line number Diff line change @@ -254,6 +254,7 @@ impl<C: ClientContext> Client<C> {
254254 Arc :: as_ptr ( & context) as * mut c_void ,
255255 )
256256 } ;
257+ native_config. set ( "log.queue" , "true" ) ?;
257258
258259 let client_ptr = unsafe {
259260 let native_config = ManuallyDrop :: new ( native_config) ;
@@ -270,6 +271,12 @@ impl<C: ClientContext> Client<C> {
270271 return Err ( KafkaError :: ClientCreation ( err_buf. to_string ( ) ) ) ;
271272 }
272273
274+ let ret = unsafe {
275+ rdsys:: rd_kafka_set_log_queue ( client_ptr, rdsys:: rd_kafka_queue_get_main ( client_ptr) )
276+ } ;
277+ if ret. is_error ( ) {
278+ return Err ( KafkaError :: Global ( ret. into ( ) ) ) ;
279+ }
273280 unsafe { rdsys:: rd_kafka_set_log_level ( client_ptr, config. log_level as i32 ) } ;
274281
275282 Ok ( Client {
Original file line number Diff line number Diff line change @@ -154,6 +154,30 @@ impl NativeClientConfig {
154154 . trim_matches ( char:: from ( 0 ) )
155155 . to_string ( ) )
156156 }
157+
158+ pub ( crate ) fn set ( & self , key : & str , value : & str ) -> KafkaResult < ( ) > {
159+ let mut err_buf = ErrBuf :: new ( ) ;
160+ let key_c = CString :: new ( key) ?;
161+ let value_c = CString :: new ( value) ?;
162+ let ret = unsafe {
163+ rdsys:: rd_kafka_conf_set (
164+ self . ptr ( ) ,
165+ key_c. as_ptr ( ) ,
166+ value_c. as_ptr ( ) ,
167+ err_buf. as_mut_ptr ( ) ,
168+ err_buf. capacity ( ) ,
169+ )
170+ } ;
171+ if ret. is_error ( ) {
172+ return Err ( KafkaError :: ClientConfig (
173+ ret,
174+ err_buf. to_string ( ) ,
175+ key. to_string ( ) ,
176+ value. to_string ( ) ,
177+ ) ) ;
178+ }
179+ Ok ( ( ) )
180+ }
157181}
158182
159183/// Client configuration.
@@ -227,27 +251,8 @@ impl ClientConfig {
227251 /// Builds a native librdkafka configuration.
228252 pub fn create_native_config ( & self ) -> KafkaResult < NativeClientConfig > {
229253 let conf = unsafe { NativeClientConfig :: from_ptr ( rdsys:: rd_kafka_conf_new ( ) ) } ;
230- let mut err_buf = ErrBuf :: new ( ) ;
231254 for ( key, value) in & self . conf_map {
232- let key_c = CString :: new ( key. to_string ( ) ) ?;
233- let value_c = CString :: new ( value. to_string ( ) ) ?;
234- let ret = unsafe {
235- rdsys:: rd_kafka_conf_set (
236- conf. ptr ( ) ,
237- key_c. as_ptr ( ) ,
238- value_c. as_ptr ( ) ,
239- err_buf. as_mut_ptr ( ) ,
240- err_buf. capacity ( ) ,
241- )
242- } ;
243- if ret. is_error ( ) {
244- return Err ( KafkaError :: ClientConfig (
245- ret,
246- err_buf. to_string ( ) ,
247- key. to_string ( ) ,
248- value. to_string ( ) ,
249- ) ) ;
250- }
255+ conf. set ( key, value) ?;
251256 }
252257 Ok ( conf)
253258 }
You can’t perform that action at this time.
0 commit comments