@@ -15,23 +15,38 @@ use crate::proto::common::rpc_sasl_proto::{SaslAuth, SaslState};
15
15
use crate :: proto:: common:: {
16
16
RpcKindProto , RpcRequestHeaderProto , RpcResponseHeaderProto , RpcSaslProto ,
17
17
} ;
18
+
18
19
use crate :: { HdfsError , Result } ;
19
20
#[ cfg( feature = "token" ) ]
20
21
use {
21
- super :: user:: Token ,
22
+ super :: user:: { BlockTokenIdentifier , Token } ,
22
23
base64:: { engine:: general_purpose, Engine as _} ,
23
24
gsasl_sys as gsasl,
24
25
libc:: { c_char, c_void, memcpy} ,
25
26
std:: ffi:: CString ,
26
27
std:: ptr,
27
28
std:: sync:: atomic:: AtomicPtr ,
28
29
} ;
30
+ #[ cfg( feature = "token" ) ]
31
+ use {
32
+ crate :: proto:: {
33
+ common:: TokenProto ,
34
+ hdfs:: {
35
+ data_transfer_encryptor_message_proto:: DataTransferEncryptorStatus , CipherOptionProto ,
36
+ CipherSuiteProto , DataTransferEncryptorMessageProto , DatanodeIdProto ,
37
+ HandshakeSecretProto ,
38
+ } ,
39
+ } ,
40
+ tokio:: io:: { AsyncBufReadExt , BufStream } ,
41
+ } ;
29
42
30
43
#[ cfg( feature = "kerberos" ) ]
31
44
use super :: gssapi:: GssapiSession ;
32
45
use super :: user:: { User , UserInfo } ;
33
46
34
47
const SASL_CALL_ID : i32 = -33 ;
48
+ #[ cfg( feature = "token" ) ]
49
+ const SASL_TRANSFER_MAGIC_NUMBER : i32 = 0xDEADBEEFu32 as i32 ;
35
50
const HDFS_DELEGATION_TOKEN : & str = "HDFS_DELEGATION_TOKEN" ;
36
51
37
52
pub ( crate ) enum AuthMethod {
@@ -309,17 +324,6 @@ impl SaslReader {
309
324
}
310
325
}
311
326
312
- // TODO: Can we implement this?
313
- // impl AsyncRead for SaslReader {
314
- // fn poll_read(
315
- // self: Pin<&mut Self>,
316
- // cx: &mut task::Context<'_>,
317
- // buf: &mut ReadBuf<'_>,
318
- // ) -> Poll<io::Result<()>> {
319
- // todo!()
320
- // }
321
- // }
322
-
323
327
pub ( crate ) struct SaslWriter {
324
328
stream : OwnedWriteHalf ,
325
329
session : Option < Arc < Mutex < Box < dyn SaslSession > > > > ,
@@ -547,3 +551,107 @@ impl Drop for GSASLSession {
547
551
}
548
552
}
549
553
}
554
+
555
+ #[ cfg( feature = "token" ) ]
556
+ pub ( crate ) struct SaslDatanodeConnection {
557
+ stream : BufStream < TcpStream > ,
558
+ }
559
+
560
+ #[ cfg( feature = "token" ) ]
561
+ impl SaslDatanodeConnection {
562
+ pub fn create ( stream : TcpStream ) -> Self {
563
+ Self {
564
+ stream : BufStream :: new ( stream) ,
565
+ }
566
+ }
567
+
568
+ pub ( crate ) async fn negotiate (
569
+ mut self ,
570
+ datanode_id : & DatanodeIdProto ,
571
+ token : & TokenProto ,
572
+ ) -> Result < TcpStream > {
573
+ // If it's a privileged port, don't do SASL negotation
574
+ if datanode_id. xfer_port <= 1024 {
575
+ return Ok ( self . stream . into_inner ( ) ) ;
576
+ }
577
+
578
+ self . stream . write_i32 ( SASL_TRANSFER_MAGIC_NUMBER ) . await ?;
579
+ self . stream . flush ( ) . await ?;
580
+
581
+ let mut session = GSASLSession :: new ( "hdfs" , "0" , & token. clone ( ) . into ( ) ) ?;
582
+
583
+ let token_identifier = BlockTokenIdentifier :: from_identifier ( & token. identifier ) ?;
584
+
585
+ let handshake_secret = if !token_identifier. handshake_secret . is_empty ( ) {
586
+ Some ( HandshakeSecretProto {
587
+ bpid : token_identifier. block_pool_id . clone ( ) ,
588
+ secret : token_identifier. handshake_secret . clone ( ) ,
589
+ } )
590
+ } else {
591
+ None
592
+ } ;
593
+
594
+ let message = DataTransferEncryptorMessageProto {
595
+ handshake_secret,
596
+ status : DataTransferEncryptorStatus :: Success as i32 ,
597
+ ..Default :: default ( )
598
+ } ;
599
+
600
+ debug ! ( "Sending data transfer encryptor message: {:?}" , message) ;
601
+
602
+ self . stream
603
+ . write_all ( & message. encode_length_delimited_to_vec ( ) )
604
+ . await ?;
605
+ self . stream . flush ( ) . await ?;
606
+
607
+ let response = self . read_sasl_response ( ) . await ?;
608
+ debug ! ( "Data transfer encryptor response: {:?}" , response) ;
609
+
610
+ let ( payload, finished) = session. step ( response. payload . as_ref ( ) . map ( |p| & p[ ..] ) ) ?;
611
+ assert ! ( !finished) ;
612
+
613
+ let message = DataTransferEncryptorMessageProto {
614
+ status : DataTransferEncryptorStatus :: Success as i32 ,
615
+ payload : Some ( payload) ,
616
+ cipher_option : vec ! [ CipherOptionProto {
617
+ suite: CipherSuiteProto :: AesCtrNopadding as i32 ,
618
+ ..Default :: default ( )
619
+ } ] ,
620
+ ..Default :: default ( )
621
+ } ;
622
+
623
+ debug ! ( "Sending data transfer encryptor message: {:?}" , message) ;
624
+
625
+ self . stream
626
+ . write_all ( & message. encode_length_delimited_to_vec ( ) )
627
+ . await ?;
628
+ self . stream . flush ( ) . await ?;
629
+
630
+ let response = self . read_sasl_response ( ) . await ?;
631
+ debug ! ( "Data transfer encryptor response: {:?}" , response) ;
632
+
633
+ let ( _, finished) = session. step ( response. payload . as_ref ( ) . map ( |p| & p[ ..] ) ) ?;
634
+
635
+ assert ! ( finished) ;
636
+
637
+ Ok ( self . stream . into_inner ( ) )
638
+ }
639
+
640
+ async fn read_sasl_response ( & mut self ) -> Result < DataTransferEncryptorMessageProto > {
641
+ self . stream . fill_buf ( ) . await ?;
642
+
643
+ let buf = self . stream . fill_buf ( ) . await ?;
644
+ if buf. is_empty ( ) {
645
+ return Err ( std:: io:: Error :: from ( std:: io:: ErrorKind :: UnexpectedEof ) ) ?;
646
+ }
647
+ let msg_length = prost:: decode_length_delimiter ( buf) ?;
648
+ let total_size = msg_length + prost:: length_delimiter_len ( msg_length) ;
649
+
650
+ let mut response_buf = BytesMut :: zeroed ( total_size) ;
651
+ self . stream . read_exact ( & mut response_buf) . await ?;
652
+
653
+ Ok ( DataTransferEncryptorMessageProto :: decode_length_delimited (
654
+ response_buf. freeze ( ) ,
655
+ ) ?)
656
+ }
657
+ }
0 commit comments