1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
use std::{
    io::{self, BufReader, BufWriter, Read, Write},
    net::{TcpStream, ToSocketAddrs},
    path::Path,
    sync::Arc,
};

use crate::{hrpc::HRpc, HDFSError};
use hdfs_types::hdfs::DatanodeIdProto;

const CLIENT_NAME: &str = "hdfs-rust-client";

mod writer;
pub use writer::{FileWriter, WriterOptions};
mod reader;
pub use reader::{FileReader, ReaderOptions};

/// HDFS 协议配置
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)]
pub struct FSConfig {
    /// name node host name or ip
    pub name_node: String,
    /// name node ipc port, default ipc port is 9000
    pub port: u16,
    /// username
    pub user: String,
}

#[derive(Debug, Clone)]
pub struct ClientConfig {
    pub real_user: Option<String>,
    pub effective_user: Option<String>,
    pub name_node: Vec<String>,
    pub connection_timeout: u64,
    pub use_hostname: bool,
    pub write_buf_size: usize,
    pub read_buf_size: usize,
    pub tcp_keepalived: Option<u64>,
    pub no_delay: bool,
}

impl Default for ClientConfig {
    fn default() -> Self {
        Self {
            real_user: Default::default(),
            effective_user: Default::default(),
            name_node: vec!["127.0.0.1:9000".into()],
            use_hostname: true,
            write_buf_size: 8192,
            read_buf_size: 8192,
            connection_timeout: 30,
            tcp_keepalived: Some(30),
            no_delay: true,
        }
    }
}

pub struct BufStream<S: Read + Write>(pub BufReader<Wrapped<S>>);

impl<S: Read + Write> BufStream<S> {
    pub fn new(stream: S) -> Self {
        Self(BufReader::new(Wrapped(BufWriter::new(stream))))
    }

    pub fn with(stream: S, read_buf: usize, write_buf: usize) -> Self {
        Self(BufReader::with_capacity(
            read_buf,
            Wrapped(BufWriter::with_capacity(write_buf, stream)),
        ))
    }
}

pub struct Wrapped<S: Read + Write>(pub BufWriter<S>);

impl<S: Read + Write> Read for Wrapped<S> {
    fn read(&mut self, buf: &mut [u8]) -> io::Result<usize> {
        self.0.get_mut().read(buf)
    }
}

impl<S: Read + Write> Write for Wrapped<S> {
    fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
        self.0.write(buf)
    }

    fn flush(&mut self) -> io::Result<()> {
        self.0.flush()
    }
}

impl<S: Read + Write> Read for BufStream<S> {
    fn read(&mut self, buf: &mut [u8]) -> io::Result<usize> {
        self.0.read(buf)
    }
}

impl<S: Read + Write> Write for BufStream<S> {
    fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
        self.0.get_mut().write(buf)
    }

    fn flush(&mut self) -> io::Result<()> {
        self.0.get_mut().flush()
    }
}

#[derive(Debug, Clone, Copy)]
pub enum IOType {
    Read,
    Write,
    Append,
}

pub struct HDFS<S: Read + Write, D: Read + Write> {
    client_name: String,
    ipc: HRpc<S>,
    create_ipc: Box<dyn Fn() -> io::Result<HRpc<S>>>,
    connect_data_node: Arc<dyn Fn(&DatanodeIdProto, IOType) -> io::Result<D> + 'static>,
}

pub trait ToNameNodes {
    fn to_name_nodes(self) -> Vec<String>;
}

impl ToNameNodes for &str {
    fn to_name_nodes(self) -> Vec<String> {
        vec![self.to_string()]
    }
}

impl ToNameNodes for Vec<String> {
    fn to_name_nodes(self) -> Vec<String> {
        self
    }
}

impl<S: ToString> ToNameNodes for &[S] {
    fn to_name_nodes(self) -> Vec<String> {
        self.iter().map(|s| s.to_string()).collect()
    }
}

impl<A: ToString, B: ToString> ToNameNodes for (A, B) {
    fn to_name_nodes(self) -> Vec<String> {
        vec![self.0.to_string(), self.1.to_string()]
    }
}

impl HDFS<BufStream<TcpStream>, BufStream<TcpStream>> {
    pub fn connect<S: ToString>(
        name_node: impl ToNameNodes,
        user: impl Into<Option<S>>,
    ) -> io::Result<Self> {
        let config = ClientConfig {
            name_node: name_node.to_name_nodes(),
            effective_user: user.into().map(|s| s.to_string()),
            ..Default::default()
        };
        Self::connect_with(config)
    }

    pub fn connect_with(config: ClientConfig) -> io::Result<Self> {
        let ClientConfig {
            real_user,
            effective_user,
            name_node,
            connection_timeout,
            use_hostname,
            write_buf_size,
            read_buf_size,
            tcp_keepalived,
            no_delay,
        } = config;
        let timeout = std::time::Duration::from_secs(connection_timeout);
        HDFS::new(
            move || {
                let stream = name_node
                    .iter()
                    .filter_map(|addr| addr.to_socket_addrs().ok())
                    .flatten()
                    .find_map(|addr| {
                        tracing::debug!(message="connect name node", addr=?addr);
                        TcpStream::connect_timeout(&addr, timeout).ok()
                    });
                let stream = stream.ok_or(io::Error::new(
                    io::ErrorKind::Other,
                    "no available name node",
                ))?;
                let sk_ref = socket2::SockRef::from(&stream);
                if let Some(keep) = tcp_keepalived {
                    let keepalive = socket2::TcpKeepalive::new()
                        .with_time(std::time::Duration::from_secs(keep));
                    sk_ref.set_tcp_keepalive(&keepalive)?;
                }
                sk_ref.set_nodelay(no_delay)?;
                let stream = BufStream::with(stream, write_buf_size, read_buf_size);
                let ipc = HRpc::connect(
                    stream,
                    effective_user.clone(),
                    real_user.clone(),
                    None,
                    None,
                )?;
                Ok(ipc)
            },
            move |datanode, _| {
                let mut addrs = if use_hostname {
                    tracing::debug!(
                        message = "connect data node",
                        hostname = datanode.host_name,
                        port = datanode.xfer_port
                    );
                    ((datanode.host_name.as_str(), datanode.xfer_port as u16))
                        .to_socket_addrs()
                        .map_err(|_| io::Error::new(io::ErrorKind::Other, "invalid address"))?
                } else {
                    tracing::debug!(
                        message = "connect data node",
                        ip = datanode.ip_addr,
                        port = datanode.xfer_port
                    );
                    ((datanode.ip_addr.as_str(), datanode.xfer_port as u16))
                        .to_socket_addrs()
                        .map_err(|_| io::Error::new(io::ErrorKind::Other, "invalid ip addr"))?
                };
                let addr = addrs
                    .next()
                    .ok_or_else(|| io::Error::new(io::ErrorKind::Other, "invalid ip address"))?;
                let stream = TcpStream::connect_timeout(&addr, timeout)?;
                let sk_ref = socket2::SockRef::from(&stream);
                if let Some(keep) = tcp_keepalived {
                    let keepalive = socket2::TcpKeepalive::new()
                        .with_time(std::time::Duration::from_secs(keep));
                    sk_ref.set_tcp_keepalive(&keepalive)?;
                }
                sk_ref.set_nodelay(no_delay)?;
                let stream = BufStream::with(stream, write_buf_size, read_buf_size);
                Ok(stream)
            },
        )
    }
}

impl<S: Read + Write, D: Read + Write> HDFS<S, D> {
    pub fn new(
        create_ipc: impl Fn() -> io::Result<HRpc<S>> + 'static,
        connect_datanode: impl Fn(&DatanodeIdProto, IOType) -> io::Result<D> + 'static,
    ) -> io::Result<Self> {
        let client_name = format!("{}_{}", CLIENT_NAME, uuid::Uuid::new_v4());
        let ipc = create_ipc()?;

        Ok(Self {
            client_name,
            ipc,
            create_ipc: Box::new(create_ipc),
            connect_data_node: Arc::new(connect_datanode),
        })
    }

    pub fn get_rpc(&mut self) -> &mut HRpc<S> {
        &mut self.ipc
    }

    pub fn new_rpc(&mut self) -> Result<HRpc<S>, io::Error> {
        (self.create_ipc)()
    }

    /// open a file
    pub fn open(&mut self, path: impl AsRef<Path>) -> Result<FileReader<D>, HDFSError> {
        ReaderOptions::default().open(path, self)
    }

    /// use reader option to custom file read options
    pub fn reader_options(&mut self) -> ReaderOptions {
        ReaderOptions::default()
    }

    /// create a file
    pub fn create(&mut self, path: impl AsRef<Path>) -> Result<FileWriter<S, D>, HDFSError> {
        WriterOptions::default().create(path, self)
    }

    /// open a existing file and append content to it
    pub fn append(&mut self, path: impl AsRef<Path>) -> Result<FileWriter<S, D>, HDFSError> {
        WriterOptions::default().append(path, self)
    }

    pub fn writer_options(&mut self) -> WriterOptions {
        WriterOptions::default()
    }

    pub fn create_dir(&mut self, path: impl AsRef<Path>) -> Result<(), HDFSError> {
        let req = hdfs_types::hdfs::MkdirsRequestProto {
            src: path.as_ref().to_string_lossy().to_string(),
            masked: hdfs_types::hdfs::FsPermissionProto { perm: 0o666 },
            create_parent: false,
            unmasked: None,
        };
        let (_, resp) = self.ipc.mkdirs(req)?;
        assert!(resp.result);
        Ok(())
    }

    pub fn create_dir_all(&mut self, path: impl AsRef<Path>) -> Result<(), HDFSError> {
        let req = hdfs_types::hdfs::MkdirsRequestProto {
            src: path.as_ref().to_string_lossy().to_string(),
            masked: hdfs_types::hdfs::FsPermissionProto { perm: 0o666 },
            create_parent: true,
            unmasked: None,
        };
        let (_, resp) = self.ipc.mkdirs(req)?;
        assert!(resp.result);
        Ok(())
    }

    /// read the entire contents of a file into a bytes vector.
    pub fn read(&mut self, path: impl AsRef<Path>) -> Result<Vec<u8>, HDFSError> {
        let mut fd = self.open(path)?;
        let mut buf = vec![0; fd.metadata().length as usize];
        fd.read_to_end(&mut buf)?;
        Ok(buf)
    }

    pub fn remote_dir(&mut self, path: impl AsRef<Path>) -> Result<(), HDFSError> {
        let req = hdfs_types::hdfs::DeleteRequestProto {
            src: path.as_ref().to_string_lossy().to_string(),
            recursive: false,
        };
        let (_, resp) = self.ipc.delete(req)?;
        assert!(resp.result);
        Ok(())
    }

    pub fn remote_dir_all(&mut self, path: impl AsRef<Path>) -> Result<(), HDFSError> {
        let req = hdfs_types::hdfs::DeleteRequestProto {
            src: path.as_ref().to_string_lossy().to_string(),
            recursive: true,
        };
        let (_, resp) = self.ipc.delete(req)?;
        assert!(resp.result);
        Ok(())
    }

    pub fn remove_file(&mut self, path: impl AsRef<Path>) -> Result<(), HDFSError> {
        let req = hdfs_types::hdfs::DeleteRequestProto {
            src: path.as_ref().to_string_lossy().to_string(),
            recursive: false,
        };
        let (_, resp) = self.ipc.delete(req)?;
        assert!(resp.result);
        Ok(())
    }

    pub fn rename(
        &mut self,
        from: impl AsRef<Path>,
        to: impl AsRef<Path>,
    ) -> Result<(), HDFSError> {
        let req = hdfs_types::hdfs::Rename2RequestProto {
            src: from.as_ref().to_string_lossy().to_string(),
            dst: to.as_ref().to_string_lossy().to_string(),
            overwrite_dest: true,
            move_to_trash: Some(true),
        };
        self.ipc.rename2(req)?;
        Ok(())
    }
}