-
Notifications
You must be signed in to change notification settings - Fork 7
/
Copy pathlib.rs
635 lines (560 loc) · 20.8 KB
/
lib.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
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
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
// #![warn(missing_docs)]
//! [object_store::ObjectStore] implementation for the Native Rust HDFS client
//!
//! # Usage
//!
//! ```rust
//! use hdfs_native_object_store::HdfsObjectStore;
//! # use object_store::Result;
//! # fn main() -> Result<()> {
//! let store = HdfsObjectStore::with_url("hdfs://localhost:9000")?;
//! # Ok(())
//! # }
//! ```
//!
use std::{
collections::HashMap,
fmt::{Display, Formatter},
future,
path::PathBuf,
sync::Arc,
};
use async_trait::async_trait;
use chrono::{DateTime, Utc};
use futures::{
stream::{BoxStream, StreamExt},
FutureExt,
};
use hdfs_native::{client::FileStatus, file::FileWriter, Client, HdfsError, WriteOptions};
use object_store::{
path::Path, GetOptions, GetRange, GetResult, GetResultPayload, ListResult, MultipartUpload,
ObjectMeta, ObjectStore, PutMode, PutMultipartOpts, PutOptions, PutPayload, PutResult, Result,
UploadPart,
};
use tokio::{
sync::{mpsc, oneshot},
task::{self, JoinHandle},
};
// Re-export minidfs for down-stream integration tests
#[cfg(feature = "integration-test")]
pub use hdfs_native::minidfs;
#[derive(Debug)]
pub struct HdfsObjectStore {
client: Arc<Client>,
}
impl HdfsObjectStore {
/// Creates a new HdfsObjectStore from an existing [Client]
///
/// ```rust
/// # use std::sync::Arc;
/// use hdfs_native::Client;
/// # use hdfs_native_object_store::HdfsObjectStore;
/// let client = Client::new("hdfs://127.0.0.1:9000").unwrap();
/// let store = HdfsObjectStore::new(Arc::new(client));
/// ```
pub fn new(client: Arc<Client>) -> Self {
Self { client }
}
/// Creates a new HdfsObjectStore using the specified URL
///
/// Connect to a NameNode
/// ```rust
/// # use hdfs_native_object_store::HdfsObjectStore;
/// # fn main() -> object_store::Result<()> {
/// let store = HdfsObjectStore::with_url("hdfs://127.0.0.1:9000")?;
/// # Ok(())
/// # }
/// ```
pub fn with_url(url: &str) -> Result<Self> {
Ok(Self::new(Arc::new(Client::new(url).to_object_store_err()?)))
}
/// Creates a new HdfsObjectStore using the specified URL and Hadoop configs.
///
/// Connect to a NameService
/// ```rust
/// # use hdfs_native_object_store::HdfsObjectStore;
/// # use std::collections::HashMap;
/// # fn main() -> object_store::Result<()> {
/// let config = HashMap::from([
/// ("dfs.ha.namenodes.ns".to_string(), "nn1,nn2".to_string()),
/// ("dfs.namenode.rpc-address.ns.nn1".to_string(), "nn1.example.com:9000".to_string()),
/// ("dfs.namenode.rpc-address.ns.nn2".to_string(), "nn2.example.com:9000".to_string()),
/// ]);
/// let store = HdfsObjectStore::with_config("hdfs://ns", config)?;
/// # Ok(())
/// # }
/// ```
pub fn with_config(url: &str, config: HashMap<String, String>) -> Result<Self> {
Ok(Self::new(Arc::new(
Client::new_with_config(url, config).to_object_store_err()?,
)))
}
async fn internal_copy(&self, from: &Path, to: &Path, overwrite: bool) -> Result<()> {
let overwrite = match self.client.get_file_info(&make_absolute_file(to)).await {
Ok(_) if overwrite => true,
Ok(_) => Err(HdfsError::AlreadyExists(make_absolute_file(to))).to_object_store_err()?,
Err(HdfsError::FileNotFound(_)) => false,
Err(e) => Err(e).to_object_store_err()?,
};
let write_options = WriteOptions {
overwrite,
..Default::default()
};
let file = self
.client
.read(&make_absolute_file(from))
.await
.to_object_store_err()?;
let mut stream = file.read_range_stream(0, file.file_length()).boxed();
let mut new_file = self
.client
.create(&make_absolute_file(to), write_options)
.await
.to_object_store_err()?;
while let Some(bytes) = stream.next().await.transpose().to_object_store_err()? {
new_file.write(bytes).await.to_object_store_err()?;
}
new_file.close().await.to_object_store_err()?;
Ok(())
}
async fn open_tmp_file(&self, file_path: &str) -> Result<(FileWriter, String)> {
let path_buf = PathBuf::from(file_path);
let file_name = path_buf
.file_name()
.ok_or(HdfsError::InvalidPath("path missing filename".to_string()))
.to_object_store_err()?
.to_str()
.ok_or(HdfsError::InvalidPath("path not valid unicode".to_string()))
.to_object_store_err()?
.to_string();
let tmp_file_path = path_buf
.with_file_name(format!(".{}.tmp", file_name))
.to_str()
.ok_or(HdfsError::InvalidPath("path not valid unicode".to_string()))
.to_object_store_err()?
.to_string();
// Try to create a file with an incrementing index until we find one that doesn't exist yet
let mut index = 1;
loop {
let path = format!("{}.{}", tmp_file_path, index);
match self.client.create(&path, WriteOptions::default()).await {
Ok(writer) => break Ok((writer, path)),
Err(HdfsError::AlreadyExists(_)) => index += 1,
Err(e) => break Err(e).to_object_store_err(),
}
}
}
}
impl Display for HdfsObjectStore {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(f, "HdfsObjectStore")
}
}
impl From<Client> for HdfsObjectStore {
fn from(value: Client) -> Self {
Self::new(Arc::new(value))
}
}
#[async_trait]
impl ObjectStore for HdfsObjectStore {
/// Save the provided bytes to the specified location
///
/// To make the operation atomic, we write to a temporary file `.{filename}.tmp.{i}` and rename
/// on a successful write, where `i` is an integer that is incremented until a non-existent file
/// is found.
async fn put_opts(
&self,
location: &Path,
payload: PutPayload,
opts: PutOptions,
) -> Result<PutResult> {
let overwrite = match opts.mode {
PutMode::Create => false,
PutMode::Overwrite => true,
PutMode::Update(_) => {
return Err(object_store::Error::NotSupported {
source: "Update mode not supported".to_string().into(),
})
}
};
let final_file_path = make_absolute_file(location);
// If we're not overwriting, do an upfront check to see if the file already
// exists. Otherwise we have to write the whole file and try to rename before
// finding out.
if !overwrite && self.client.get_file_info(&final_file_path).await.is_ok() {
return Err(HdfsError::AlreadyExists(final_file_path)).to_object_store_err();
}
let (mut tmp_file, tmp_file_path) = self.open_tmp_file(&final_file_path).await?;
for bytes in payload {
tmp_file.write(bytes).await.to_object_store_err()?;
}
tmp_file.close().await.to_object_store_err()?;
self.client
.rename(&tmp_file_path, &final_file_path, overwrite)
.await
.to_object_store_err()?;
Ok(PutResult {
e_tag: None,
version: None,
})
}
/// Create a multipart writer that writes to a temporary file in a background task, and renames
/// to the final destination on complete.
async fn put_multipart_opts(
&self,
location: &Path,
_opts: PutMultipartOpts,
) -> Result<Box<dyn MultipartUpload>> {
let final_file_path = make_absolute_file(location);
let (tmp_file, tmp_file_path) = self.open_tmp_file(&final_file_path).await?;
Ok(Box::new(HdfsMultipartWriter::new(
Arc::clone(&self.client),
tmp_file,
&tmp_file_path,
&final_file_path,
)))
}
/// Reads data for the specified location.
async fn get_opts(&self, location: &Path, options: GetOptions) -> Result<GetResult> {
if options.if_match.is_some()
|| options.if_none_match.is_some()
|| options.if_modified_since.is_some()
|| options.if_unmodified_since.is_some()
{
return Err(object_store::Error::NotImplemented);
}
let meta = self.head(location).await?;
let range = options
.range
.map(|r| match r {
GetRange::Bounded(range) => range,
GetRange::Offset(offset) => offset..meta.size,
GetRange::Suffix(suffix) => meta.size.saturating_sub(suffix)..meta.size,
})
.unwrap_or(0..meta.size);
let reader = self
.client
.read(&make_absolute_file(location))
.await
.to_object_store_err()?;
let stream = reader
.read_range_stream(range.start, range.end - range.start)
.map(|b| b.to_object_store_err())
.boxed();
let payload = GetResultPayload::Stream(stream);
Ok(GetResult {
payload,
meta,
range,
attributes: Default::default(),
})
}
/// Return the metadata for the specified location
async fn head(&self, location: &Path) -> Result<ObjectMeta> {
let status = self
.client
.get_file_info(&make_absolute_file(location))
.await
.to_object_store_err()?;
if status.isdir {
return Err(HdfsError::IsADirectoryError(
"Head must be called on a file".to_string(),
))
.to_object_store_err();
}
Ok(ObjectMeta {
location: location.clone(),
last_modified: DateTime::<Utc>::from_timestamp(status.modification_time as i64, 0)
.unwrap(),
size: status.length,
e_tag: None,
version: None,
})
}
/// Delete the object at the specified location.
async fn delete(&self, location: &Path) -> Result<()> {
let result = self
.client
.delete(&make_absolute_file(location), false)
.await
.to_object_store_err()?;
if !result {
Err(HdfsError::OperationFailed(
"failed to delete object".to_string(),
))
.to_object_store_err()?
}
Ok(())
}
/// List all the objects with the given prefix.
///
/// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of
/// `foo/bar_baz/x`.
///
/// Note: the order of returned [`ObjectMeta`] is not guaranteed
fn list(&self, prefix: Option<&Path>) -> BoxStream<'_, Result<ObjectMeta>> {
let status_stream = self
.client
.list_status_iter(
&prefix.map(make_absolute_dir).unwrap_or("".to_string()),
true,
)
.into_stream()
.filter(|res| {
let result = match res {
Ok(status) => !status.isdir,
// Listing by prefix should just return an empty list if the prefix isn't found
Err(HdfsError::FileNotFound(_)) => false,
_ => true,
};
future::ready(result)
})
.map(|res| res.map_or_else(|e| Err(e).to_object_store_err(), |s| get_object_meta(&s)));
Box::pin(status_stream)
}
/// List objects with the given prefix and an implementation specific
/// delimiter. Returns common prefixes (directories) in addition to object
/// metadata.
///
/// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of
/// `foo/bar_baz/x`.
async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result<ListResult> {
let mut status_stream = self
.client
.list_status_iter(
&prefix.map(make_absolute_dir).unwrap_or("".to_string()),
false,
)
.into_stream()
.filter(|res| {
let result = match res {
// Listing by prefix should just return an empty list if the prefix isn't found
Err(HdfsError::FileNotFound(_)) => false,
_ => true,
};
future::ready(result)
});
let mut statuses = Vec::<FileStatus>::new();
while let Some(status) = status_stream.next().await {
statuses.push(status.to_object_store_err()?);
}
let mut dirs: Vec<Path> = Vec::new();
for status in statuses.iter().filter(|s| s.isdir) {
dirs.push(Path::parse(&status.path)?)
}
let mut files: Vec<ObjectMeta> = Vec::new();
for status in statuses.iter().filter(|s| !s.isdir) {
files.push(get_object_meta(status)?)
}
Ok(ListResult {
common_prefixes: dirs,
objects: files,
})
}
/// Renames a file. This operation is guaranteed to be atomic.
async fn rename(&self, from: &Path, to: &Path) -> Result<()> {
Ok(self
.client
.rename(&make_absolute_file(from), &make_absolute_file(to), true)
.await
.to_object_store_err()?)
}
/// Renames a file only if the distination doesn't exist. This operation is guaranteed
/// to be atomic.
async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> {
Ok(self
.client
.rename(&make_absolute_file(from), &make_absolute_file(to), false)
.await
.to_object_store_err()?)
}
/// Copy an object from one path to another in the same object store.
///
/// If there exists an object at the destination, it will be overwritten.
async fn copy(&self, from: &Path, to: &Path) -> Result<()> {
self.internal_copy(from, to, true).await
}
/// Copy an object from one path to another, only if destination is empty.
///
/// Will return an error if the destination already has an object.
///
/// Performs an atomic operation if the underlying object storage supports it.
/// If atomic operations are not supported by the underlying object storage (like S3)
/// it will return an error.
async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> {
self.internal_copy(from, to, false).await
}
}
#[cfg(feature = "integration-test")]
pub trait HdfsErrorConvert<T> {
fn to_object_store_err(self) -> Result<T>;
}
#[cfg(not(feature = "integration-test"))]
trait HdfsErrorConvert<T> {
fn to_object_store_err(self) -> Result<T>;
}
impl<T> HdfsErrorConvert<T> for hdfs_native::Result<T> {
fn to_object_store_err(self) -> Result<T> {
self.map_err(|err| match err {
HdfsError::FileNotFound(path) => object_store::Error::NotFound {
path: path.clone(),
source: Box::new(HdfsError::FileNotFound(path)),
},
HdfsError::AlreadyExists(path) => object_store::Error::AlreadyExists {
path: path.clone(),
source: Box::new(HdfsError::AlreadyExists(path)),
},
_ => object_store::Error::Generic {
store: "HdfsObjectStore",
source: Box::new(err),
},
})
}
}
type PartSender = mpsc::UnboundedSender<(oneshot::Sender<Result<()>>, PutPayload)>;
// Create a fake multipart writer the creates an uploader to a temp file as a background
// task, and submits new parts to be uploaded to a queue for this task.
// A once cell is used to track whether a part has finished writing or not.
// On completing, rename the file to the actual target.
struct HdfsMultipartWriter {
client: Arc<Client>,
sender: Option<(JoinHandle<Result<()>>, PartSender)>,
tmp_filename: String,
final_filename: String,
}
impl HdfsMultipartWriter {
fn new(
client: Arc<Client>,
writer: FileWriter,
tmp_filename: &str,
final_filename: &str,
) -> Self {
let (sender, receiver) = mpsc::unbounded_channel();
Self {
client,
sender: Some((Self::start_writer_task(writer, receiver), sender)),
tmp_filename: tmp_filename.to_string(),
final_filename: final_filename.to_string(),
}
}
fn start_writer_task(
mut writer: FileWriter,
mut part_receiver: mpsc::UnboundedReceiver<(oneshot::Sender<Result<()>>, PutPayload)>,
) -> JoinHandle<Result<()>> {
task::spawn(async move {
'outer: loop {
match part_receiver.recv().await {
Some((sender, part)) => {
for bytes in part {
if let Err(e) = writer.write(bytes).await.to_object_store_err() {
let _ = sender.send(Err(e));
break 'outer;
}
}
let _ = sender.send(Ok(()));
}
None => {
return writer.close().await.to_object_store_err();
}
}
}
// If we've reached here, a write task failed so just return Err's for all new parts that come in
while let Some((sender, _)) = part_receiver.recv().await {
let _ = sender.send(
Err(HdfsError::OperationFailed(
"Write failed during one of the parts".to_string(),
))
.to_object_store_err(),
);
}
Err(HdfsError::OperationFailed(
"Write failed during one of the parts".to_string(),
))
.to_object_store_err()
})
}
}
impl std::fmt::Debug for HdfsMultipartWriter {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("HdfsMultipartWriter")
.field("tmp_filename", &self.tmp_filename)
.field("final_filename", &self.final_filename)
.finish()
}
}
#[async_trait]
impl MultipartUpload for HdfsMultipartWriter {
fn put_part(&mut self, payload: PutPayload) -> UploadPart {
let (result_sender, result_receiver) = oneshot::channel();
if let Some((_, payload_sender)) = self.sender.as_ref() {
payload_sender.send((result_sender, payload)).unwrap();
} else {
result_sender
.send(
Err(HdfsError::OperationFailed(
"Cannot put part after completing or aborting".to_string(),
))
.to_object_store_err(),
)
.unwrap();
}
async { result_receiver.await.unwrap() }.boxed()
}
async fn complete(&mut self) -> Result<PutResult> {
// Drop the sender so the task knows no more data is coming
if let Some((handle, sender)) = self.sender.take() {
drop(sender);
// Wait for the writer task to finish
handle.await??;
self.client
.rename(&self.tmp_filename, &self.final_filename, true)
.await
.to_object_store_err()?;
Ok(PutResult {
e_tag: None,
version: None,
})
} else {
Err(object_store::Error::NotSupported {
source: "Cannot call abort or complete multiple times".into(),
})
}
}
async fn abort(&mut self) -> Result<()> {
// Drop the sender so the task knows no more data is coming
if let Some((handle, sender)) = self.sender.take() {
drop(sender);
// Wait for the writer task to finish
handle.abort();
self.client
.delete(&self.tmp_filename, false)
.await
.to_object_store_err()?;
Ok(())
} else {
Err(object_store::Error::NotSupported {
source: "Cannot call abort or complete multiple times".into(),
})
}
}
}
/// ObjectStore paths always remove the leading slash, so add it back
fn make_absolute_file(path: &Path) -> String {
format!("/{}", path.as_ref())
}
fn make_absolute_dir(path: &Path) -> String {
if path.parts().count() > 0 {
format!("/{}/", path.as_ref())
} else {
"/".to_string()
}
}
fn get_object_meta(status: &FileStatus) -> Result<ObjectMeta> {
Ok(ObjectMeta {
location: Path::parse(&status.path)?,
last_modified: DateTime::<Utc>::from_timestamp(status.modification_time as i64, 0).unwrap(),
size: status.length,
e_tag: None,
version: None,
})
}