opendal/services/hdfs_native/
backend.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use std::fmt::Debug;
19use std::fmt::Formatter;
20use std::sync::Arc;
21
22use log::debug;
23
24use super::HDFS_NATIVE_SCHEME;
25use super::core::HdfsNativeCore;
26use super::delete::HdfsNativeDeleter;
27use super::error::parse_hdfs_error;
28use super::lister::HdfsNativeLister;
29use super::reader::HdfsNativeReader;
30use super::writer::HdfsNativeWriter;
31use crate::raw::*;
32use crate::services::HdfsNativeConfig;
33use crate::*;
34/// [Hadoop Distributed File System (HDFS™)](https://hadoop.apache.org/) support.
35/// Using [Native Rust HDFS client](https://github.com/Kimahriman/hdfs-native).
36#[doc = include_str!("docs.md")]
37#[derive(Default)]
38pub struct HdfsNativeBuilder {
39    pub(super) config: HdfsNativeConfig,
40}
41
42impl Debug for HdfsNativeBuilder {
43    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
44        f.debug_struct("HdfsNativeBuilder")
45            .field("config", &self.config)
46            .finish()
47    }
48}
49
50impl HdfsNativeBuilder {
51    /// Set root of this backend.
52    ///
53    /// All operations will happen under this root.
54    pub fn root(mut self, root: &str) -> Self {
55        self.config.root = if root.is_empty() {
56            None
57        } else {
58            Some(root.to_string())
59        };
60
61        self
62    }
63
64    /// Set name_node of this backend.
65    ///
66    /// Valid format including:
67    ///
68    /// - `default`: using the default setting based on hadoop config.
69    /// - `hdfs://127.0.0.1:9000`: connect to hdfs cluster.
70    pub fn name_node(mut self, name_node: &str) -> Self {
71        if !name_node.is_empty() {
72            // Trim trailing `/` so that we can accept `http://127.0.0.1:9000/`
73            self.config.name_node = Some(name_node.trim_end_matches('/').to_string())
74        }
75
76        self
77    }
78
79    /// Enable append capacity of this backend.
80    ///
81    /// This should be disabled when HDFS runs in non-distributed mode.
82    pub fn enable_append(mut self, enable_append: bool) -> Self {
83        self.config.enable_append = enable_append;
84        self
85    }
86}
87
88impl Builder for HdfsNativeBuilder {
89    type Config = HdfsNativeConfig;
90
91    fn build(self) -> Result<impl Access> {
92        debug!("backend build started: {:?}", &self);
93
94        let name_node = match &self.config.name_node {
95            Some(v) => v,
96            None => {
97                return Err(Error::new(ErrorKind::ConfigInvalid, "name_node is empty")
98                    .with_context("service", Scheme::HdfsNative));
99            }
100        };
101
102        let root = normalize_root(&self.config.root.unwrap_or_default());
103        debug!("backend use root {root}");
104
105        let client = hdfs_native::ClientBuilder::new()
106            .with_url(name_node)
107            .build()
108            .map_err(parse_hdfs_error)?;
109
110        // need to check if root dir exists, create if not
111        Ok(HdfsNativeBackend {
112            core: Arc::new(HdfsNativeCore {
113                info: {
114                    let am = AccessorInfo::default();
115                    am.set_scheme(HDFS_NATIVE_SCHEME)
116                        .set_root(&root)
117                        .set_native_capability(Capability {
118                            stat: true,
119
120                            read: true,
121
122                            write: true,
123                            write_can_append: self.config.enable_append,
124
125                            create_dir: true,
126                            delete: true,
127
128                            list: true,
129
130                            rename: true,
131
132                            shared: true,
133
134                            ..Default::default()
135                        });
136
137                    am.into()
138                },
139                root,
140                client: Arc::new(client),
141                enable_append: self.config.enable_append,
142            }),
143        })
144    }
145}
146
147// #[inline]
148// fn tmp_file_of(path: &str) -> String {
149//     let name = get_basename(path);
150//     let uuid = Uuid::new_v4().to_string();
151
152//     format!("{name}.{uuid}")
153// }
154
155/// Backend for hdfs-native services.
156#[derive(Debug, Clone)]
157pub struct HdfsNativeBackend {
158    core: Arc<HdfsNativeCore>,
159}
160
161impl Access for HdfsNativeBackend {
162    type Reader = HdfsNativeReader;
163    type Writer = HdfsNativeWriter;
164    type Lister = Option<HdfsNativeLister>;
165    type Deleter = oio::OneShotDeleter<HdfsNativeDeleter>;
166
167    fn info(&self) -> Arc<AccessorInfo> {
168        self.core.info.clone()
169    }
170
171    async fn create_dir(&self, path: &str, _args: OpCreateDir) -> Result<RpCreateDir> {
172        self.core.hdfs_create_dir(path).await?;
173        Ok(RpCreateDir::default())
174    }
175
176    async fn stat(&self, path: &str, _args: OpStat) -> Result<RpStat> {
177        let m = self.core.hdfs_stat(path).await?;
178        Ok(RpStat::new(m))
179    }
180
181    async fn read(&self, path: &str, args: OpRead) -> Result<(RpRead, Self::Reader)> {
182        let (f, offset, size) = self.core.hdfs_read(path, &args).await?;
183
184        let r = HdfsNativeReader::new(f, offset as _, size as _);
185
186        Ok((RpRead::new(), r))
187    }
188
189    async fn write(&self, path: &str, args: OpWrite) -> Result<(RpWrite, Self::Writer)> {
190        let (f, initial_size) = self.core.hdfs_write(path, &args).await?;
191
192        Ok((RpWrite::new(), HdfsNativeWriter::new(f, initial_size)))
193    }
194
195    async fn delete(&self) -> Result<(RpDelete, Self::Deleter)> {
196        Ok((
197            RpDelete::default(),
198            oio::OneShotDeleter::new(HdfsNativeDeleter::new(Arc::clone(&self.core))),
199        ))
200    }
201
202    async fn list(&self, path: &str, _args: OpList) -> Result<(RpList, Self::Lister)> {
203        match self.core.hdfs_list(path).await? {
204            Some((p, current_path)) => Ok((
205                RpList::default(),
206                Some(HdfsNativeLister::new(
207                    &self.core.root,
208                    &self.core.client,
209                    &p,
210                    current_path,
211                )),
212            )),
213            None => Ok((RpList::default(), None)),
214        }
215    }
216
217    async fn rename(&self, from: &str, to: &str, _args: OpRename) -> Result<RpRename> {
218        self.core.hdfs_rename(from, to).await?;
219        Ok(RpRename::default())
220    }
221}