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

[#4993] feat(iceberg): integrate credential framework to iceberg REST server #5134

Merged
merged 18 commits into from
Oct 18, 2024

Conversation

FANNG1
Copy link
Contributor

@FANNG1 FANNG1 commented Oct 15, 2024

What changes were proposed in this pull request?

integrate credential framework to iceberg REST server

Why are the changes needed?

Fix: #4993

Does this PR introduce any user-facing change?

No

How was this patch tested?

set up a local environment to request credential with the configuration spark.sql.catalog.rest.header.X-Iceberg-Access-Delegation=vended-credentials

@FANNG1 FANNG1 marked this pull request as draft October 15, 2024 03:39
@FANNG1 FANNG1 changed the title [SIP] integrate credential framework to iceberg REST server [#4993] feat(iceberg): integrate credential framework to iceberg REST server Oct 15, 2024
@FANNG1 FANNG1 marked this pull request as ready for review October 15, 2024 10:57
@FANNG1
Copy link
Contributor Author

FANNG1 commented Oct 15, 2024

@jerryshao , it's ready for review now, please help to review when you are free, thanks

public static final String CREDENTIAL_TYPE = "credential-type";

private CredentialConstants() {}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

I remembered that we already defined this in Credential, we don't have to define this again here.

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 not the same, one is the credential type in Credential , this is the credential type in catalog properties. They are happen to the same name.

Copy link
Contributor

Choose a reason for hiding this comment

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

Why can't you use a different name?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

use credential-provider-type

Copy link
Contributor

Choose a reason for hiding this comment

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

If it is a catalog property, it is better to define in catalog-common, so other engines can also use it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

the property already defined in catalog-common

@@ -110,7 +110,7 @@ public void initialize(
resultConf.put("catalog_uuid", info.id().toString());
IcebergConfig icebergConfig = new IcebergConfig(resultConf);

this.icebergCatalogWrapper = new IcebergCatalogWrapper(icebergConfig);
this.icebergCatalogWrapper = new IcebergCatalogWrapper(icebergConfig, false);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you add a comment here to explain the reason why we hard code the false here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

// todo: transform specific credential to iceberg properties
return credential.toProperties();
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do we add this Iceberg specific method in common module, can it be in iceberg-common module?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This will also be used in client or spark to transform Credential properties to the properties used for Iceberg. so I place it in common.

Comment on lines -107 to -108
public IcebergCatalogWrapper() {
this(new IcebergConfig(Collections.emptyMap()));
Copy link
Contributor

Choose a reason for hiding this comment

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

We can still maintain this default constructor like:

public IcebergCatalogWrapper() {
  this(xxxx, false)
}

What do you think?

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 used to test only, seems useless to maintain the default constructor

// todo(fanng): check user privilege.
PathBasedCredentialContext pathBasedCredentialContext =
new PathBasedCredentialContext(
PrincipalUtils.getCurrentUserName(), ImmutableSet.of(location), ImmutableSet.of());
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you need to set write path?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, because we don't know user will write or read after loading table. We have to give the write privilege to the user .After we supporting ACL, we could grant write privilege to the user only with write privilege.

return catalogConfigToClients;
}

private Map<String, String> vendCredentials(String location) {
// ifPresentOrElse is not supported in Java8
if (credentialProvider.isPresent()) {
Copy link
Contributor

@jerryshao jerryshao Oct 16, 2024

Choose a reason for hiding this comment

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

You can change to simplify your code here:

if (!xx.isPresent) {
  throw new XXXException();
}

....

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@@ -75,7 +75,7 @@ public IcebergCatalogWrapper getIcebergTableOps(String catalogName) {
LOG.warn(errorMsg);
throw new RuntimeException(errorMsg);
}
return new IcebergCatalogWrapper(icebergConfig);
return new IcebergCatalogWrapper(icebergConfig, true);
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do we need to have a boolean to control whether to use credential vending or not, is there any better way?

Also, I saw there're lot of hard-code value in this PR, if it is enabled by default, maybe we can figure out a better way other than hard-coded boolean.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

removed credential vending logic from IcebergCatalogWrapper, do credential vending after related operation is finished outside.

LOG.info(
"Create Iceberg table, namespace: {}, create table request: {}",
"Create Iceberg table, namespace: {}, create table request: {}, accessDelegation: {}, isCredentialVending:{}",
Copy link
Contributor

Choose a reason for hiding this comment

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

Add space before {}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@FANNG1 FANNG1 marked this pull request as draft October 17, 2024 06:15
}

@Test
void testCreateTableWithCredentialVending() {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

add credential vending test here

@FANNG1 FANNG1 marked this pull request as ready for review October 17, 2024 09:40
@FANNG1
Copy link
Contributor Author

FANNG1 commented Oct 17, 2024

@jerryshao please help to review again

Comment on lines +34 to +36
public static Map<String, String> toIcebergProperties(Credential credential) {
// todo: transform specific credential to iceberg properties
return credential.toProperties();
Copy link
Contributor

Choose a reason for hiding this comment

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

This two lines code is not worthy to create a new class here, why can't you put this in iceberg package?

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 like a placeholder function to support more credential properties transform logic, this function will be used in client side and Iceberg REST server side, so I place it in common module.


public void registerCredentialProvider(
String catalogName, CredentialProvider credentialProvider) {
CredentialProvider current = credentialProviders.putIfAbsent(catalogName, credentialProvider);
Copy link
Contributor

Choose a reason for hiding this comment

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

Do we support one catalog with multiple credential vendors? If so, seems this map cannot support it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not in current PR, I prefer to support it later in integrate Credential vending to Gravitino server to have a better overall design.

@@ -40,11 +39,12 @@
* gravitino.iceberg-rest.catalog.hive_proxy.catalog-backend = hive
* gravitino.iceberg-rest.catalog.hive_proxy.uri = thrift://{host}:{port} ...
*/
public class ConfigBasedIcebergCatalogWrapperProvider implements IcebergCatalogWrapperProvider {
public class ConfigBasedIcebergCatalogConfigProvider implements IcebergCatalogConfigProvider {
Copy link
Contributor

Choose a reason for hiding this comment

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

The class name is a little tricky, is there a better name for it?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

FileBasedIcebergCatalogConfigProvider or StaticIcebergCatalogConfigProvider ?

Copy link
Contributor

Choose a reason for hiding this comment

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

FileBasedXXX and CatalogBasedXXX, how about this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

CatalogBasedCatalogConfigProvider seems a little odd

Copy link
Contributor Author

Choose a reason for hiding this comment

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

use StaticCatalogConfigProvider and DynamicCatalogConfigProvider , cc @theoryxu

@HeaderParam(X_ICEBERG_ACCESS_DELEGATION) String accessDelegation) {
boolean isCredentialVending = isCredentialVending(accessDelegation);
LOG.info(
"Load iceberg table, namespace: {}, table: {}, accessDelegation: {}, is credential vending: {}",
Copy link
Contributor

@jerryshao jerryshao Oct 18, 2024

Choose a reason for hiding this comment

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

access delegation: {}..., credential vending: {}

Preconditions.checkState(
!credentialProvider.equals(current),
String.format(
"Should not register multi times to CredentialProviderManager, catalog:%s, credential provider:%s",
Copy link
Contributor

Choose a reason for hiding this comment

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

"multiple times to..."

X_ICEBERG_ACCESS_DELEGATION
+ ": "
+ accessDelegation
+ " is illegal, Iceberg REST spec supports:[vended-credentials,remote-signing], Gravitino Iceberg REST server supports: vended-credentials");
Copy link
Contributor

Choose a reason for hiding this comment

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

This line is too long, it should be no longer than 100 chars.

X_ICEBERG_ACCESS_DELEGATION
+ ": "
+ accessDelegation
+ " is illegal, Iceberg REST spec supports:[vended-credentials,remote-signing], Gravitino Iceberg REST server supports: vended-credentials");
Copy link
Contributor

Choose a reason for hiding this comment

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

if (xxx.isBlank) {
  return false;
}

if (xxx.equals...) {
  return true;
}

if (xxx.equals...) {
  throw xxx;
}

throw xxx;
}

jerryshao
jerryshao previously approved these changes Oct 18, 2024
@jerryshao
Copy link
Contributor

Please fix the build issue here.

@FANNG1
Copy link
Contributor Author

FANNG1 commented Oct 18, 2024

Please fix the build issue here.

updated, please help to review again

@jerryshao jerryshao merged commit 10e2943 into apache:main Oct 18, 2024
26 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Subtask] integrate credential vending with IcebergRESTServer
2 participants