hdfs_native/
file.rs

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
use std::sync::Arc;
use std::time::Duration;

use bytes::{BufMut, Bytes, BytesMut};
use futures::stream::BoxStream;
use futures::{stream, Stream, StreamExt};
use log::warn;

use crate::ec::{resolve_ec_policy, EcSchema};
use crate::hdfs::block_reader::get_block_stream;
use crate::hdfs::block_writer::BlockWriter;
use crate::hdfs::protocol::{LeaseTracker, NamenodeProtocol};
use crate::proto::hdfs;
use crate::{HdfsError, Result};

const COMPLETE_RETRY_DELAY_MS: u64 = 500;
const COMPLETE_RETRIES: u32 = 5;

pub struct FileReader {
    protocol: Arc<NamenodeProtocol>,
    status: hdfs::HdfsFileStatusProto,
    located_blocks: hdfs::LocatedBlocksProto,
    ec_schema: Option<EcSchema>,
    position: usize,
}

impl FileReader {
    pub(crate) fn new(
        protocol: Arc<NamenodeProtocol>,
        status: hdfs::HdfsFileStatusProto,
        located_blocks: hdfs::LocatedBlocksProto,
        ec_schema: Option<EcSchema>,
    ) -> Self {
        Self {
            protocol,
            status,
            located_blocks,
            ec_schema,
            position: 0,
        }
    }

    /// Returns the total size of the file
    pub fn file_length(&self) -> usize {
        self.status.length as usize
    }

    /// Returns the remaining bytes left based on the current cursor position.
    pub fn remaining(&self) -> usize {
        if self.position > self.file_length() {
            0
        } else {
            self.file_length() - self.position
        }
    }

    /// Sets the cursor to the position. Panics if the position is beyond the end of the file
    pub fn seek(&mut self, pos: usize) {
        if pos > self.file_length() {
            panic!("Cannot seek beyond the end of a file");
        }
        self.position = pos;
    }

    /// Returns the current cursor position in the file
    pub fn tell(&self) -> usize {
        self.position
    }

    /// Read up to `len` bytes into a new [Bytes] object, advancing the internal position in the file.
    /// An empty [Bytes] object will be returned if the end of the file has been reached.
    pub async fn read(&mut self, len: usize) -> Result<Bytes> {
        if self.position >= self.file_length() {
            Ok(Bytes::new())
        } else {
            let offset = self.position;
            self.position = usize::min(self.position + len, self.file_length());
            self.read_range(offset, self.position - offset).await
        }
    }

    /// Read up to `buf.len()` bytes into the provided slice, advancing the internal position in the file.
    /// Returns the number of bytes that were read, or 0 if the end of the file has been reached.
    pub async fn read_buf(&mut self, buf: &mut [u8]) -> Result<usize> {
        if self.position >= self.file_length() {
            Ok(0)
        } else {
            let offset = self.position;
            self.position = usize::min(self.position + buf.len(), self.file_length());
            let read_bytes = self.position - offset;
            self.read_range_buf(buf, offset).await?;
            Ok(read_bytes)
        }
    }

    /// Read up to `len` bytes starting at `offset` into a new [Bytes] object. The returned buffer
    /// could be smaller than `len` if `offset + len` extends beyond the end of the file.
    ///
    /// Panics if the requested range is outside of the file
    pub async fn read_range(&self, offset: usize, len: usize) -> Result<Bytes> {
        let mut stream = self.read_range_stream(offset, len).boxed();
        let mut buf = BytesMut::with_capacity(len);
        while let Some(bytes) = stream.next().await.transpose()? {
            buf.put(bytes);
        }
        Ok(buf.freeze())
    }

    /// Read file data into an existing buffer
    ///
    /// Panics if the requested range is outside of the file
    pub async fn read_range_buf(&self, mut buf: &mut [u8], offset: usize) -> Result<()> {
        let mut stream = self.read_range_stream(offset, buf.len()).boxed();
        while let Some(bytes) = stream.next().await.transpose()? {
            buf.put(bytes);
        }

        Ok(())
    }

    /// Return a stream of `Bytes` objects containing the content of the file
    ///
    /// Panics if the requested range is outside of the file
    pub fn read_range_stream(
        &self,
        offset: usize,
        len: usize,
    ) -> impl Stream<Item = Result<Bytes>> {
        if offset + len > self.file_length() {
            panic!("Cannot read past end of the file");
        }

        let block_streams: Vec<BoxStream<Result<Bytes>>> = self
            .located_blocks
            .blocks
            .iter()
            .flat_map(move |block| {
                let block_file_start = block.offset as usize;
                let block_file_end = block_file_start + block.b.num_bytes() as usize;

                if block_file_start < (offset + len) && block_file_end > offset {
                    // We need to read this block
                    let block_start = offset - usize::min(offset, block_file_start);
                    let block_end = usize::min(offset + len, block_file_end) - block_file_start;
                    Some(get_block_stream(
                        Arc::clone(&self.protocol),
                        block.clone(),
                        block_start,
                        block_end - block_start,
                        self.ec_schema.clone(),
                    ))
                } else {
                    // No data is needed from this block
                    None
                }
            })
            .collect();

        stream::iter(block_streams).flatten()
    }
}

pub struct FileWriter {
    src: String,
    protocol: Arc<NamenodeProtocol>,
    status: hdfs::HdfsFileStatusProto,
    block_writer: Option<BlockWriter>,
    last_block: Option<hdfs::LocatedBlockProto>,
    closed: bool,
    bytes_written: usize,
}

impl FileWriter {
    pub(crate) fn new(
        protocol: Arc<NamenodeProtocol>,
        src: String,
        status: hdfs::HdfsFileStatusProto,
        // Some for append, None for create
        last_block: Option<hdfs::LocatedBlockProto>,
    ) -> Self {
        protocol.add_file_lease(status.file_id(), status.namespace.clone());
        Self {
            protocol,
            src,
            status,
            block_writer: None,
            last_block,
            closed: false,
            bytes_written: 0,
        }
    }

    async fn create_block_writer(&mut self) -> Result<()> {
        let new_block = if let Some(last_block) = self.last_block.take() {
            // Append operation on first write. Erasure code appends always just create a new block.
            if last_block.b.num_bytes() < self.status.blocksize() && self.status.ec_policy.is_none()
            {
                // The last block isn't full, just write data to it
                last_block
            } else {
                // The last block is full, so create a new block to write to
                self.protocol
                    .add_block(&self.src, Some(last_block.b), self.status.file_id)
                    .await?
                    .block
            }
        } else {
            // Not appending to an existing block, just create a new one
            // If there's an existing block writer, close it first
            let extended_block = if let Some(block_writer) = self.block_writer.take() {
                let extended_block = block_writer.get_extended_block();
                block_writer.close().await?;
                Some(extended_block)
            } else {
                None
            };

            self.protocol
                .add_block(&self.src, extended_block, self.status.file_id)
                .await?
                .block
        };

        let block_writer = BlockWriter::new(
            Arc::clone(&self.protocol),
            new_block,
            self.status.blocksize() as usize,
            self.protocol.get_cached_server_defaults().await?,
            self.status
                .ec_policy
                .as_ref()
                .map(resolve_ec_policy)
                .transpose()?
                .as_ref(),
        )
        .await?;

        self.block_writer = Some(block_writer);
        Ok(())
    }

    async fn get_block_writer(&mut self) -> Result<&mut BlockWriter> {
        // If the current writer is full, or hasn't been created, create one
        if self.block_writer.as_ref().is_some_and(|b| b.is_full()) || self.block_writer.is_none() {
            self.create_block_writer().await?;
        }

        Ok(self.block_writer.as_mut().unwrap())
    }

    pub async fn write(&mut self, mut buf: Bytes) -> Result<usize> {
        let bytes_to_write = buf.len();
        // Create a shallow copy of the bytes instance to mutate and track what's been read
        while !buf.is_empty() {
            let block_writer = self.get_block_writer().await?;

            block_writer.write(&mut buf).await?;
        }

        self.bytes_written += bytes_to_write;

        Ok(bytes_to_write)
    }

    pub async fn close(&mut self) -> Result<()> {
        if !self.closed {
            let extended_block = if let Some(block_writer) = self.block_writer.take() {
                let extended_block = block_writer.get_extended_block();
                block_writer.close().await?;
                Some(extended_block)
            } else {
                None
            };

            let mut retry_delay = COMPLETE_RETRY_DELAY_MS;
            let mut retries = 0;
            while retries < COMPLETE_RETRIES {
                let successful = self
                    .protocol
                    .complete(&self.src, extended_block.clone(), self.status.file_id)
                    .await?
                    .result;

                if successful {
                    self.closed = true;
                    return Ok(());
                }

                tokio::time::sleep(Duration::from_millis(retry_delay)).await;

                retry_delay *= 2;
                retries += 1;
            }
            Err(HdfsError::OperationFailed(
                "Failed to complete file in time".to_string(),
            ))
        } else {
            Ok(())
        }
    }
}

impl Drop for FileWriter {
    fn drop(&mut self) {
        if !self.closed {
            warn!("FileWriter dropped without being closed. File content may not have saved or may not be complete");
        }

        self.protocol
            .remove_file_lease(self.status.file_id(), self.status.namespace.clone());
    }
}