Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: implement OAuth for catalog rest client #254

Merged
merged 6 commits into from
Mar 18, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
118 changes: 113 additions & 5 deletions crates/catalog/rest/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ use iceberg::{

use self::_serde::{
CatalogConfig, ErrorResponse, ListNamespaceResponse, ListTableResponse, NamespaceSerde,
RenameTableRequest, NO_CONTENT, OK,
RenameTableRequest, TokenResponse, NO_CONTENT, OK,
};

const ICEBERG_REST_SPEC_VERSION: &str = "0.14.1";
Expand Down Expand Up @@ -96,9 +96,13 @@ impl RestCatalogConfig {
.join("/")
}

fn get_token_endpoint(&self) -> String {
[&self.uri, PATH_V1, "oauth", "tokens"].join("/")
}

fn try_create_rest_client(&self) -> Result<HttpClient> {
//TODO: We will add oauth, ssl config, sigv4 later
let headers = HeaderMap::from_iter([
// TODO: We will add ssl config, sigv4 later
let mut headers = HeaderMap::from_iter([
(
header::CONTENT_TYPE,
HeaderValue::from_static("application/json"),
Expand All @@ -113,6 +117,19 @@ impl RestCatalogConfig {
),
]);

if let Some(token) = self.props.get("token") {
liurenjie1024 marked this conversation as resolved.
Show resolved Hide resolved
headers.insert(
header::AUTHORIZATION,
HeaderValue::from_str(&format!("Bearer {token}")).map_err(|e| {
Error::new(
ErrorKind::DataInvalid,
"Invalid token received from catalog server!",
)
.with_source(e)
})?,
);
}

Ok(HttpClient(
Client::builder().default_headers(headers).build()?,
))
Expand Down Expand Up @@ -144,13 +161,15 @@ impl HttpClient {
.with_source(e)
})?)
} else {
let code = resp.status();
let text = resp.bytes().await?;
let e = serde_json::from_slice::<E>(&text).map_err(|e| {
Error::new(
ErrorKind::Unexpected,
"Failed to parse response from rest catalog server!",
)
.with_context("json", String::from_utf8_lossy(&text))
.with_context("code", code.to_string())
liurenjie1024 marked this conversation as resolved.
Show resolved Hide resolved
.with_source(e)
})?;
Err(e.into())
Expand Down Expand Up @@ -497,13 +516,56 @@ impl RestCatalog {
client: config.try_create_rest_client()?,
config,
};

catalog.fetch_access_token().await?;
catalog.client = catalog.config.try_create_rest_client()?;
Comment on lines +519 to +520
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The python implementation update config first, then recreate client? Is this required or we can relax? cc @Fokko @flyrain

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's necessary. For example, in the tabular's hosted iceberg, they will return a new uri to override your config. It's necessary to recreate a new http client.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I don't get your point. I know that we should recreate http client, what I mean is that is it required that update_config happens before fetch_access_token? Seems java does this in reverse order, so I guess it doesn't matter.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what I mean is that is it required that update_config happens before fetch_access_token?

fetch_access_token happens before update_config in my code, and also in the python code.

Some config API also need authentication, such as https://api.tabular.io/ws/v1/config

catalog.update_config().await?;
catalog.client = catalog.config.try_create_rest_client()?;

Ok(catalog)
}

async fn fetch_access_token(&mut self) -> Result<()> {
TennyZhuang marked this conversation as resolved.
Show resolved Hide resolved
if self.config.props.contains_key("token") {
return Ok(());
}
liurenjie1024 marked this conversation as resolved.
Show resolved Hide resolved
if let Some(credential) = self.config.props.get("credential") {
let (client_id, client_secret) = if credential.contains(':') {
let (client_id, client_secret) = credential.split_once(':').unwrap();
(Some(client_id), client_secret)
} else {
(None, credential.as_str())
};
let mut params = HashMap::with_capacity(4);
params.insert("grant_type", "client_credentials");
if let Some(client_id) = client_id {
params.insert("client_id", client_id);
}
params.insert("client_secret", client_secret);
params.insert("scope", "catalog");
TennyZhuang marked this conversation as resolved.
Show resolved Hide resolved
let req = self
.client
.0
.post(self.config.get_token_endpoint())
TennyZhuang marked this conversation as resolved.
Show resolved Hide resolved
.form(&params)
.build()?;
let res = self
.client
.query::<TokenResponse, ErrorResponse, OK>(req)
.await
.map_err(|e| {
Error::new(
ErrorKind::Unexpected,
"Failed to fetch access token from catalog server!",
)
.with_source(e)
})?;
let token = res.access_token;
self.config.props.insert("token".to_string(), token);
}

Ok(())
}

async fn update_config(&mut self) -> Result<()> {
let mut request = self.client.0.get(self.config.config_endpoint());

Expand Down Expand Up @@ -626,6 +688,14 @@ mod _serde {
}
}

#[derive(Debug, Serialize, Deserialize)]
pub(super) struct TokenResponse {
pub(super) access_token: String,
pub(super) token_type: String,
pub(super) expires_in: Option<u64>,
pub(super) issued_token_type: Option<String>,
}

#[derive(Debug, Serialize, Deserialize)]
pub(super) struct NamespaceSerde {
pub(super) namespace: Vec<String>,
Expand Down Expand Up @@ -778,6 +848,44 @@ mod tests {
.await
}

async fn create_oauth_mock(server: &mut ServerGuard) -> Mock {
server
.mock("POST", "/v1/oauth/tokens")
.with_status(200)
.with_body(
r#"{
"access_token": "ey000000000000",
"token_type": "Bearer",
"issued_token_type": "urn:ietf:params:oauth:token-type:access_token",
"expires_in": 86400
}"#,
)
.create_async()
.await
}

#[tokio::test]
async fn test_oauth() {
let mut server = Server::new_async().await;
let oauth_mock = create_oauth_mock(&mut server).await;
let config_mock = create_config_mock(&mut server).await;

let mut props = HashMap::new();
props.insert("credential".to_string(), "client1:secret1".to_string());

let _catalog = RestCatalog::new(
RestCatalogConfig::builder()
.uri(server.url())
.props(props)
.build(),
)
.await
.unwrap();

oauth_mock.assert_async().await;
config_mock.assert_async().await;
}

#[tokio::test]
async fn test_list_namespace() {
let mut server = Server::new_async().await;
Expand Down Expand Up @@ -1557,7 +1665,7 @@ mod tests {
"type": "NoSuchTableException",
"code": 404
}
}
}
"#,
)
.create_async()
Expand Down
1 change: 1 addition & 0 deletions crates/catalog/rest/tests/rest_catalog_test.rs
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ async fn set_test_fixture(func: &str) -> TestFixture {
rest_catalog,
}
}

#[tokio::test]
async fn test_get_non_exist_namespace() {
let fixture = set_test_fixture("test_get_non_exist_namespace").await;
Expand Down
Loading