-
Notifications
You must be signed in to change notification settings - Fork 9
/
Copy pathhdfs.rs
831 lines (707 loc) · 26.6 KB
/
hdfs.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
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! Object store that represents the HDFS File System.
use std::collections::{BTreeSet, VecDeque};
use std::fmt::{Display, Formatter};
use std::ops::Range;
use std::path::PathBuf;
use std::sync::{Arc, Mutex};
use async_trait::async_trait;
use bytes::Bytes;
use chrono::{DateTime, NaiveDateTime, Utc};
use futures::{stream::BoxStream, StreamExt, TryStreamExt};
use hdfs::hdfs::{get_hdfs_by_full_path, FileStatus, HdfsErr, HdfsFile, HdfsFs};
use hdfs::walkdir::HdfsWalkDir;
use object_store::multipart::{CloudMultiPartUpload, CloudMultiPartUploadImpl};
use object_store::{
path::{self, Path},
Error, GetOptions, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, Result,
};
use tokio::io::AsyncWrite;
/// scheme for HDFS File System
pub static HDFS_SCHEME: &str = "hdfs";
/// scheme for HDFS Federation File System
pub static VIEWFS_SCHEME: &str = "viewfs";
#[derive(Debug)]
/// Hadoop File System as Object Store.
pub struct HadoopFileSystem {
hdfs: Arc<HdfsFs>,
}
impl Default for HadoopFileSystem {
fn default() -> Self {
Self {
hdfs: get_hdfs_by_full_path("default").expect("Fail to get default HdfsFs"),
}
}
}
impl HadoopFileSystem {
/// Get HDFS from the full path, like hdfs://localhost:8020/xxx/xxx
pub fn new(full_path: &str) -> Option<Self> {
get_hdfs_by_full_path(full_path)
.map(|hdfs| Some(Self { hdfs }))
.unwrap_or(None)
}
/// Return filesystem path of the given location
fn path_to_filesystem(location: &Path) -> String {
format!("/{}", location.as_ref())
}
pub fn get_path_root(&self) -> String {
self.hdfs.url().to_owned()
}
pub fn get_path(&self, full_path: &str) -> Path {
get_path(full_path, self.hdfs.url())
}
pub fn get_hdfs_host(&self) -> String {
let hdfs_url = self.hdfs.url();
if hdfs_url.starts_with(HDFS_SCHEME) {
hdfs_url[7..].to_owned()
} else if hdfs_url.starts_with(VIEWFS_SCHEME) {
hdfs_url[9..].to_owned()
} else {
"".to_owned()
}
}
fn read_range(range: &Range<usize>, file: &HdfsFile) -> Result<Bytes> {
let to_read = range.end - range.start;
let mut buf = vec![0; to_read];
let read = file
.read_with_pos(range.start as i64, buf.as_mut_slice())
.map_err(to_error)?;
assert_eq!(
to_read as i32,
read,
"Read path {} from {} with expected size {} and actual size {}",
file.path(),
range.start,
to_read,
read
);
Ok(buf.into())
}
}
impl Display for HadoopFileSystem {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
write!(f, "HadoopFileSystem")
}
}
struct HdfsMultiPartUpload {
location: Path,
hdfs: Arc<HdfsFs>,
content: Arc<Mutex<Vec<Option<Vec<u8>>>>>,
first_unwritten_idx: Arc<Mutex<usize>>,
file_created: Arc<Mutex<bool>>,
}
impl HdfsMultiPartUpload {
fn create_file_if_necessary(&self) -> Result<()> {
let mut file_created = self.file_created.lock().unwrap();
if !*file_created {
let location = HadoopFileSystem::path_to_filesystem(&self.location.clone());
match self.hdfs.create_with_overwrite(&location, true) {
Ok(_) => {
*file_created = true;
Ok(())
}
Err(e) => Err(to_error(e)),
}
} else {
Ok(())
}
}
}
#[async_trait]
impl CloudMultiPartUploadImpl for HdfsMultiPartUpload {
async fn put_multipart_part(
&self,
buf: Vec<u8>,
part_idx: usize,
) -> Result<object_store::multipart::UploadPart, std::io::Error> {
{
let mut content = self.content.lock().unwrap();
while content.len() <= part_idx {
content.push(None);
}
content[part_idx] = Some(buf);
}
let location = HadoopFileSystem::path_to_filesystem(&self.location.clone());
let first_unwritten_idx = {
let guard = self.first_unwritten_idx.lock().unwrap();
*guard
};
self.create_file_if_necessary()?;
// Attempt to write all contiguous sequences of parts
if first_unwritten_idx <= part_idx {
let hdfs = self.hdfs.clone();
let content = self.content.clone();
let first_unwritten_idx = self.first_unwritten_idx.clone();
maybe_spawn_blocking(move || {
let file = hdfs.append(&location).map_err(to_error)?;
let mut content = content.lock().unwrap();
let mut first_unwritten_idx = first_unwritten_idx.lock().unwrap();
// Write all contiguous parts and free up the memory
while let Some(buf) = content.get_mut(*first_unwritten_idx).and_then(Option::take) {
file.write(buf.as_slice()).map_err(to_error)?;
*first_unwritten_idx += 1;
}
file.close().map_err(to_error)?;
Ok(())
})
.await
.map_err(to_io_error)?;
}
Ok(object_store::multipart::UploadPart {
content_id: part_idx.to_string(),
})
}
async fn complete(
&self,
completed_parts: Vec<object_store::multipart::UploadPart>,
) -> Result<(), std::io::Error> {
let content = self.content.lock().unwrap();
if content.len() != completed_parts.len() {
return Err(std::io::Error::new(
std::io::ErrorKind::InvalidData,
format!(
"Expected {} parts, but only {} parts were received",
content.len(),
completed_parts.len()
),
));
}
// check first_unwritten_idx
let first_unwritten_idx = self.first_unwritten_idx.lock().unwrap();
if *first_unwritten_idx != content.len() {
return Err(std::io::Error::new(
std::io::ErrorKind::InvalidData,
format!(
"Expected to write {} parts, but only {} parts were written",
content.len(),
*first_unwritten_idx
),
));
}
// Last check: make sure all parts were written, since we change it to None after writing
if content.iter().any(Option::is_some) {
return Err(std::io::Error::new(
std::io::ErrorKind::InvalidData,
"Not all parts were written",
));
}
Ok(())
}
}
#[async_trait]
impl ObjectStore for HadoopFileSystem {
// Current implementation is very simple due to missing configs,
// like whether able to overwrite, whether able to create parent directories, etc
async fn put(&self, location: &Path, bytes: Bytes) -> Result<()> {
let hdfs = self.hdfs.clone();
let location = HadoopFileSystem::path_to_filesystem(location);
maybe_spawn_blocking(move || {
let file = match hdfs.create_with_overwrite(&location, true) {
Ok(f) => f,
Err(e) => {
return Err(to_error(e));
}
};
file.write(bytes.as_ref()).map_err(to_error)?;
file.close().map_err(to_error)?;
Ok(())
})
.await
}
async fn put_multipart(
&self,
location: &Path,
) -> Result<(MultipartId, Box<dyn AsyncWrite + Unpin + Send>)> {
let upload = HdfsMultiPartUpload {
location: location.clone(),
hdfs: self.hdfs.clone(),
content: Arc::new(Mutex::new(Vec::new())),
first_unwritten_idx: Arc::new(Mutex::new(0)),
file_created: Arc::new(Mutex::new(false)),
};
Ok((
MultipartId::default(),
Box::new(CloudMultiPartUpload::new(upload, 8)),
))
}
async fn abort_multipart(&self, location: &Path, _multipart_id: &MultipartId) -> Result<()> {
// remove the file if it exists
self.delete(location).await
}
async fn get(&self, location: &Path) -> Result<GetResult> {
let hdfs = self.hdfs.clone();
let location = HadoopFileSystem::path_to_filesystem(location);
let blob: Bytes = maybe_spawn_blocking(move || {
let file = hdfs.open(&location).map_err(to_error)?;
let file_status = file.get_file_status().map_err(to_error)?;
let to_read = file_status.len();
let mut buf = vec![0; to_read];
let read = file.read(buf.as_mut_slice()).map_err(to_error)?;
assert_eq!(
to_read as i32, read,
"Read path {} with expected size {} and actual size {}",
&location, to_read, read
);
file.close().map_err(to_error)?;
Ok(buf.into())
})
.await?;
Ok(GetResult::Stream(
futures::stream::once(async move { Ok(blob) }).boxed(),
))
}
async fn get_opts(&self, location: &Path, options: GetOptions) -> Result<GetResult> {
if options.if_match.is_some() || options.if_none_match.is_some() {
return Err(Error::Generic {
store: "HadoopFileSystem",
source: Box::new(HdfsErr::Generic("ETags not supported".to_string())),
});
}
let hdfs = self.hdfs.clone();
let location = HadoopFileSystem::path_to_filesystem(location);
let blob: Bytes = maybe_spawn_blocking(move || {
let file = hdfs.open(&location).map_err(to_error)?;
let file_status = file.get_file_status().map_err(to_error)?;
if options.if_unmodified_since.is_some() || options.if_modified_since.is_some() {
check_modified(&options, &location, last_modified(&file_status))?;
}
let range = if let Some(range) = options.range {
range
} else {
Range {
start: 0,
end: file_status.len(),
}
};
let buf = Self::read_range(&range, &file)?;
file.close().map_err(to_error)?;
Ok(buf)
})
.await?;
Ok(GetResult::Stream(
futures::stream::once(async move { Ok(blob) }).boxed(),
))
}
async fn get_range(&self, location: &Path, range: Range<usize>) -> Result<Bytes> {
let hdfs = self.hdfs.clone();
let location = HadoopFileSystem::path_to_filesystem(location);
maybe_spawn_blocking(move || {
let file = hdfs.open(&location).map_err(to_error)?;
let buf = Self::read_range(&range, &file)?;
file.close().map_err(to_error)?;
Ok(buf)
})
.await
}
async fn get_ranges(&self, location: &Path, ranges: &[Range<usize>]) -> Result<Vec<Bytes>> {
coalesce_ranges(
ranges,
|range| self.get_range(location, range),
HDFS_COALESCE_DEFAULT,
)
.await
}
async fn head(&self, location: &Path) -> Result<ObjectMeta> {
let hdfs = self.hdfs.clone();
let hdfs_root = self.hdfs.url().to_owned();
let location = HadoopFileSystem::path_to_filesystem(location);
maybe_spawn_blocking(move || {
let file_status = hdfs.get_file_status(&location).map_err(to_error)?;
Ok(convert_metadata(file_status, &hdfs_root))
})
.await
}
async fn delete(&self, location: &Path) -> Result<()> {
let hdfs = self.hdfs.clone();
let location = HadoopFileSystem::path_to_filesystem(location);
maybe_spawn_blocking(move || {
hdfs.delete(&location, false).map_err(to_error)?;
Ok(())
})
.await
}
/// List all of the leaf files under the prefix path.
/// It will recursively search leaf files whose depth is larger than 1
async fn list(&self, prefix: Option<&Path>) -> Result<BoxStream<'_, Result<ObjectMeta>>> {
let default_path = Path::from(self.get_path_root());
let prefix = prefix.unwrap_or(&default_path);
let hdfs = self.hdfs.clone();
let hdfs_root = self.hdfs.url().to_owned();
let walkdir =
HdfsWalkDir::new_with_hdfs(HadoopFileSystem::path_to_filesystem(prefix), hdfs)
.min_depth(1);
let s =
walkdir.into_iter().flat_map(move |result_dir_entry| {
match convert_walkdir_result(result_dir_entry) {
Err(e) => Some(Err(e)),
Ok(None) => None,
Ok(entry @ Some(_)) => entry
.filter(|dir_entry| dir_entry.is_file())
.map(|entry| Ok(convert_metadata(entry, &hdfs_root))),
}
});
// If no tokio context, return iterator directly as no
// need to perform chunked spawn_blocking reads
if tokio::runtime::Handle::try_current().is_err() {
return Ok(futures::stream::iter(s).boxed());
}
// Otherwise list in batches of CHUNK_SIZE
const CHUNK_SIZE: usize = 1024;
let buffer = VecDeque::with_capacity(CHUNK_SIZE);
let stream = futures::stream::try_unfold((s, buffer), |(mut s, mut buffer)| async move {
if buffer.is_empty() {
(s, buffer) = tokio::task::spawn_blocking(move || {
for _ in 0..CHUNK_SIZE {
match s.next() {
Some(r) => buffer.push_back(r),
None => break,
}
}
(s, buffer)
})
.await?;
}
match buffer.pop_front() {
Some(Err(e)) => Err(e),
Some(Ok(meta)) => Ok(Some((meta, (s, buffer)))),
None => Ok(None),
}
});
Ok(stream.boxed())
}
/// List files and directories directly under the prefix path.
/// It will not recursively search leaf files whose depth is larger than 1
async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result<ListResult> {
let default_path = Path::from(self.get_path_root());
let prefix = prefix.unwrap_or(&default_path);
let hdfs = self.hdfs.clone();
let hdfs_root = self.hdfs.url().to_owned();
let walkdir =
HdfsWalkDir::new_with_hdfs(HadoopFileSystem::path_to_filesystem(prefix), hdfs)
.min_depth(1)
.max_depth(1);
let prefix = prefix.clone();
maybe_spawn_blocking(move || {
let mut common_prefixes = BTreeSet::new();
let mut objects = Vec::new();
for entry_res in walkdir.into_iter().map(convert_walkdir_result) {
if let Some(entry) = entry_res? {
let is_directory = entry.is_directory();
let entry_location = get_path(entry.name(), &hdfs_root);
let mut parts = match entry_location.prefix_match(&prefix) {
Some(parts) => parts,
None => continue,
};
let common_prefix = match parts.next() {
Some(p) => p,
None => continue,
};
drop(parts);
if is_directory {
common_prefixes.insert(prefix.child(common_prefix));
} else {
objects.push(convert_metadata(entry, &hdfs_root));
}
}
}
Ok(ListResult {
common_prefixes: common_prefixes.into_iter().collect(),
objects,
})
})
.await
}
/// Copy an object from one path to another.
/// If there exists an object at the destination, it will be overwritten.
async fn copy(&self, from: &Path, to: &Path) -> Result<()> {
let hdfs = self.hdfs.clone();
let from = HadoopFileSystem::path_to_filesystem(from);
let to = HadoopFileSystem::path_to_filesystem(to);
maybe_spawn_blocking(move || {
// We need to make sure the source exist
if !hdfs.exist(&from) {
return Err(Error::NotFound {
path: from.clone(),
source: Box::new(HdfsErr::FileNotFound(from)),
});
}
// Delete destination if exists
if hdfs.exist(&to) {
hdfs.delete(&to, false).map_err(to_error)?;
}
hdfs::util::HdfsUtil::copy(hdfs.as_ref(), &from, hdfs.as_ref(), &to)
.map_err(to_error)?;
Ok(())
})
.await
}
/// It's only allowed for the same HDFS
async fn rename(&self, from: &Path, to: &Path) -> Result<()> {
let hdfs = self.hdfs.clone();
let from = HadoopFileSystem::path_to_filesystem(from);
let to = HadoopFileSystem::path_to_filesystem(to);
maybe_spawn_blocking(move || {
hdfs.rename(&from, &to).map_err(to_error)?;
Ok(())
})
.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.
async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> {
let hdfs = self.hdfs.clone();
let from = HadoopFileSystem::path_to_filesystem(from);
let to = HadoopFileSystem::path_to_filesystem(to);
maybe_spawn_blocking(move || {
if hdfs.exist(&to) {
return Err(Error::AlreadyExists {
path: from,
source: Box::new(HdfsErr::FileAlreadyExists(to)),
});
}
hdfs::util::HdfsUtil::copy(hdfs.as_ref(), &from, hdfs.as_ref(), &to)
.map_err(to_error)?;
Ok(())
})
.await
}
}
/// Create Path without prefix
pub fn get_path(full_path: &str, prefix: &str) -> Path {
let partial_path = &full_path[prefix.len()..];
Path::parse(partial_path).unwrap()
}
/// Convert HDFS file status to ObjectMeta
pub fn convert_metadata(file: FileStatus, prefix: &str) -> ObjectMeta {
ObjectMeta {
location: get_path(file.name(), prefix),
last_modified: last_modified(&file),
size: file.len(),
e_tag: None,
}
}
fn last_modified(file: &FileStatus) -> DateTime<Utc> {
DateTime::<Utc>::from_utc(
NaiveDateTime::from_timestamp_opt(file.last_modified(), 0).unwrap(),
Utc,
)
}
fn check_modified(
get_options: &GetOptions,
location: &str,
last_modified: DateTime<Utc>,
) -> Result<()> {
if let Some(date) = get_options.if_modified_since {
if last_modified <= date {
return Err(Error::NotModified {
path: location.to_string(),
source: format!("{} >= {}", date, last_modified).into(),
});
}
}
if let Some(date) = get_options.if_unmodified_since {
if last_modified > date {
return Err(Error::Precondition {
path: location.to_string(),
source: format!("{} < {}", date, last_modified).into(),
});
}
}
Ok(())
}
/// Convert walkdir results and converts not-found errors into `None`.
fn convert_walkdir_result(
res: std::result::Result<FileStatus, HdfsErr>,
) -> Result<Option<FileStatus>> {
match res {
Ok(entry) => Ok(Some(entry)),
Err(walkdir_err) => match walkdir_err {
HdfsErr::FileNotFound(_) => Ok(None),
_ => Err(to_error(HdfsErr::Generic(
"Fail to walk hdfs directory".to_owned(),
))),
},
}
}
/// Range requests with a gap less than or equal to this,
/// will be coalesced into a single request by [`coalesce_ranges`]
pub const HDFS_COALESCE_DEFAULT: usize = 1024 * 1024;
/// Up to this number of range requests will be performed in parallel by [`coalesce_ranges`]
pub const OBJECT_STORE_COALESCE_PARALLEL: usize = 10;
/// Takes a function to fetch ranges and coalesces adjacent ranges if they are
/// less than `coalesce` bytes apart.
pub async fn coalesce_ranges<F, Fut>(
ranges: &[Range<usize>],
fetch: F,
coalesce: usize,
) -> Result<Vec<Bytes>>
where
F: FnMut(Range<usize>) -> Fut,
Fut: std::future::Future<Output = Result<Bytes>>,
{
let fetch_ranges = merge_ranges(ranges, coalesce);
let fetched: Vec<_> = futures::stream::iter(fetch_ranges.iter().cloned())
.map(fetch)
.buffered(OBJECT_STORE_COALESCE_PARALLEL)
.try_collect()
.await?;
Ok(ranges
.iter()
.map(|range| {
let idx = fetch_ranges.partition_point(|v| v.start <= range.start) - 1;
let fetch_range = &fetch_ranges[idx];
let fetch_bytes = &fetched[idx];
let start = range.start - fetch_range.start;
let end = range.end - fetch_range.start;
fetch_bytes.slice(start..end)
})
.collect())
}
/// Takes a function and spawns it to a tokio blocking pool if available
pub async fn maybe_spawn_blocking<F, T>(f: F) -> Result<T>
where
F: FnOnce() -> Result<T> + Send + 'static,
T: Send + 'static,
{
#[cfg(feature = "try_spawn_blocking")]
match tokio::runtime::Handle::try_current() {
Ok(runtime) => runtime.spawn_blocking(f).await?,
Err(_) => f(),
}
#[cfg(not(feature = "try_spawn_blocking"))]
f()
}
/// Returns a sorted list of ranges that cover `ranges`
fn merge_ranges(ranges: &[Range<usize>], coalesce: usize) -> Vec<Range<usize>> {
if ranges.is_empty() {
return vec![];
}
let mut ranges = ranges.to_vec();
ranges.sort_unstable_by_key(|range| range.start);
let mut ret = Vec::with_capacity(ranges.len());
let mut start_idx = 0;
let mut end_idx = 1;
while start_idx != ranges.len() {
let mut range_end = ranges[start_idx].end;
while end_idx != ranges.len()
&& ranges[end_idx]
.start
.checked_sub(range_end)
.map(|delta| delta <= coalesce)
.unwrap_or(true)
{
range_end = range_end.max(ranges[end_idx].end);
end_idx += 1;
}
let start = ranges[start_idx].start;
let end = range_end;
ret.push(start..end);
start_idx = end_idx;
end_idx += 1;
}
ret
}
fn to_error(err: HdfsErr) -> Error {
match err {
HdfsErr::FileNotFound(path) => Error::NotFound {
path: path.clone(),
source: Box::new(HdfsErr::FileNotFound(path)),
},
HdfsErr::FileAlreadyExists(path) => Error::AlreadyExists {
path: path.clone(),
source: Box::new(HdfsErr::FileAlreadyExists(path)),
},
HdfsErr::InvalidUrl(path) => Error::InvalidPath {
source: path::Error::InvalidPath {
path: PathBuf::from(path),
},
},
HdfsErr::CannotConnectToNameNode(namenode_uri) => Error::Generic {
store: "HadoopFileSystem",
source: Box::new(HdfsErr::CannotConnectToNameNode(namenode_uri)),
},
HdfsErr::Generic(err_str) => Error::Generic {
store: "HadoopFileSystem",
source: Box::new(HdfsErr::Generic(err_str)),
},
}
}
fn to_io_error(err: Error) -> std::io::Error {
match err {
Error::Generic { store, source } => {
std::io::Error::new(std::io::ErrorKind::Other, format!("{}: {}", store, source))
}
Error::NotFound { path, source } => std::io::Error::new(
std::io::ErrorKind::NotFound,
format!("{}: {}", path, source),
),
Error::AlreadyExists { path, source } => std::io::Error::new(
std::io::ErrorKind::AlreadyExists,
format!("{}: {}", path, source),
),
Error::InvalidPath { source } => {
std::io::Error::new(std::io::ErrorKind::InvalidInput, source)
}
_ => std::io::Error::new(
std::io::ErrorKind::Other,
format!("HadoopFileSystem: {}", err),
),
}
}
#[cfg(test)]
mod tests {
use super::*;
use std::ops::Range;
#[tokio::test]
async fn test_coalesce_ranges() {
let do_fetch = |ranges: Vec<Range<usize>>, coalesce: usize| async move {
let max = ranges.iter().map(|x| x.end).max().unwrap_or(0);
let src: Vec<_> = (0..max).map(|x| x as u8).collect();
let mut fetches = vec![];
let coalesced = coalesce_ranges(
&ranges,
|range| {
fetches.push(range.clone());
futures::future::ready(Ok(Bytes::from(src[range].to_vec())))
},
coalesce,
)
.await
.unwrap();
assert_eq!(ranges.len(), coalesced.len());
for (range, bytes) in ranges.iter().zip(coalesced) {
assert_eq!(bytes.as_ref(), &src[range.clone()]);
}
fetches
};
let fetches = do_fetch(vec![], 0).await;
assert_eq!(fetches, vec![]);
let fetches = do_fetch(vec![0..3], 0).await;
assert_eq!(fetches, vec![0..3]);
let fetches = do_fetch(vec![0..2, 3..5], 0).await;
assert_eq!(fetches, vec![0..2, 3..5]);
let fetches = do_fetch(vec![0..1, 1..2], 0).await;
assert_eq!(fetches, vec![0..2]);
let fetches = do_fetch(vec![0..1, 2..72], 1).await;
assert_eq!(fetches, vec![0..72]);
let fetches = do_fetch(vec![0..1, 56..72, 73..75], 1).await;
assert_eq!(fetches, vec![0..1, 56..75]);
let fetches = do_fetch(vec![0..1, 5..6, 7..9, 4..6], 1).await;
assert_eq!(fetches, vec![0..1, 4..9]);
}
}