-
Notifications
You must be signed in to change notification settings - Fork 230
Implement federation to HadoopCatalog #1466
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
Implement federation to HadoopCatalog #1466
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM! Left few minor comments!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we name the package as rest
? Also, IcebergCatalogProperties
is also implemented by HadoopConnectionConfigInfoDpo
, it's used by any Iceberg related connection configs, maybe we can leave it in the current package (iceberg
).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My concern with that alternative is that delegation to e.g. Unity Catalog will also be over REST. So it's more like iceberg-rest, I called it iceberg
but rest
is not ideal either
|
||
public HadoopConnectionConfigInfoDpo( | ||
@JsonProperty(value = "uri", required = true) @Nonnull String uri, | ||
@JsonProperty(value = "authenticationParameters", required = true) @Nonnull |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
QQ: What type of authentication method does HadoopCatalog
typically use? I tried looking for some docs but couldn't find much.
Should we add a no-op authentication method?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We essentially just forward whatever properties into the catalog itself, which I think functions as the auth mechanism
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we extend or refactor HadoopCatalog
then the auth could be more properly injectable; unfortunately right now it appears to just hard-code using HadoopFileIO
as its fileIOImpl
and also independently instantiates a org.apache.hadoop.fs.FileSystem
, which relies on the org.apache.hadoop.conf.Configuration
which appears to be null in this PR since we call the no-arg constructor of HadoopCatalog
instead of the 2-arg constructor that takes a Configuration
.
From what I can tell this means it'll just fallback to Hadoop's "default" Configuration loading where it would normally load things from e.g. hdfs-site.xml
.
If we're going to really support running this federation mode properly, we should at least add a TODO to expand a HadoopCatalog-specific AuthenticationParameters subclass. One way would be to express similar delegation settings as StorageConfigInfo
, or alternatively, HadoopCatalog is a special-case where the real storage credential is the catalog credential, so maybe we could just have something that "points" at the Catalog's StorageConfigInfo as the effective AuthenticationParameters.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I actually think supporting HadoopCatalog as a sort of "dumb" federation mode that just uses the properties you give it (and loads from hdfs-site.xml
by default) is fine... if we wanted to add another module that brings in a POLARIS_HADOOP
mode or something in the future that could be nice, too.
In practice, the way you would probably deploy this is by creating a second Polaris instance dedicated to wrapping the Hadoop catalog, where you can specify env variables and a hdfs-site.xml
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it appears to just hard-code using HadoopFileIO
I think the properties we pass into initialize
should take effect right? That would be the fallback if io-impl
is not specified.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we restrict HadoopCatalog backed by cloud-providers only (like S3) and not HDFS clusters for phase 1, as opening up HDFS will add more cases like HDFS backed by kreberos, its not just hdfs-site.xml that Configuration contains, thoughts ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the concept of federation means that Polaris itself is completely agnostic to the configuration of the remote catalog. Otherwise, when we delegate to IRCs, should we give admins some kind of knobs about which IRCs we can and cannot delegate to? Should we enforce certain settings for those IRCs?
In my view, it makes sense to just create a HadoopCatalog and pass along whatever properties the user provides when the user has configured Polaris to delegate to a Hadoop catalog.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In my view, it makes sense to just create a HadoopCatalog and pass along whatever properties the user provides when the user has configured Polaris to delegate to a Hadoop catalog.
I agree in principal for this, I was mostly coming from introducing / opening up for some use-case that polaris doesn't support in the moment itself, hence was my consideration, but totally fine if we move ahead.
I think the concept of federation means that Polaris itself is completely agnostic to the configuration of the remote catalog.
I am not sure if I fully agree with this though, my reasoning is following because Polaris can't just be a pass through, the identity that the federated catalog sees is much more powerful so Polaris should IMHO need to some down scoping or never the less make a way to provide the federated catalog the requester's identity, which in Kreberos world is way different and have to go hand in had with downstream catalog so some co-ordination is required. mostly talking of this : #1026 (comment)
May be I am making it more complicated, than its supposed to be at-least this is chain of thoughts that come to my mind, on thinking about this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Polaris should IMHO need to some down scoping or never the less make a way to provide the federated catalog the requester's identity
I agree with you that this should be a goal. I would point out however that the same would be true for even the simplest case -- Polaris -> Polaris federation. This problem isn't unique to federation to Hadoop. It is, however, a very hard problem. What if the two systems don't have the same concept of identity, or they don't trust the same IdP?
There are cases where identity federation may unlock new solutions for us, where the two catalogs can federate to the same trusted IdP. But there are also going to be cases where this is not so.
The way Polaris -> IRC federation currently works, and the way that I think federation will have to continue working for the time being, is that Polaris simply communicates with the remote catalog via the connection config. This can be set on the catalog level today, but in the future we will be able to set this on the namespace or the table level, similar to a storage config.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Right, identity-passthrough/delegation is important long-term, but has a lot of complicated prerequisites and other dependencies on how we express federated identities and user-isolated caching of metadata or storage credentials, so in the meantime we should just clearly document what the "powerful connectionconfig" model implies.
@@ -32,6 +32,7 @@ | |||
public enum ConnectionType { | |||
NULL_TYPE(0), | |||
ICEBERG_REST(1), | |||
HADOOP(2), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we plan to federate to Hadoop
through a non-Iceberg Catalog
?
Just curious how generic tables would work with Catalog Federation in that case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is for Hadoop's IcebergCatalog
, but good point. For UC, we might have a separate connection for UC's IRC endpoints compared to the non-IRC ones and they might be used for different table types
allOf: | ||
- $ref: '#/components/schemas/ConnectionConfigInfo' | ||
properties: | ||
warehouse: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just want to flag that, for IcebergRestConnectionConfigInfo
, we use remoteCatalogName
, and it's passed as the warehouse property when initializing the catalog client. Using warehouse
make more sense for HadoopConnectionConfigInfo
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This seems like a good starting point for simple HadoopCatalog use cases where the storage credentials can be broadly shared in the Polaris server's runtime environment, but we'll probably want to expand on the authentication parameters in a followup to provide more configurability and guardrails around more explicitly specifying storage roles/credentials.
In the meantime, we'll need the ability to specify which connection config types to allow at the service-level so that different connection types can be independently enabled/disabled.
|
||
public HadoopConnectionConfigInfoDpo( | ||
@JsonProperty(value = "uri", required = true) @Nonnull String uri, | ||
@JsonProperty(value = "authenticationParameters", required = true) @Nonnull |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we extend or refactor HadoopCatalog
then the auth could be more properly injectable; unfortunately right now it appears to just hard-code using HadoopFileIO
as its fileIOImpl
and also independently instantiates a org.apache.hadoop.fs.FileSystem
, which relies on the org.apache.hadoop.conf.Configuration
which appears to be null in this PR since we call the no-arg constructor of HadoopCatalog
instead of the 2-arg constructor that takes a Configuration
.
From what I can tell this means it'll just fallback to Hadoop's "default" Configuration loading where it would normally load things from e.g. hdfs-site.xml
.
If we're going to really support running this federation mode properly, we should at least add a TODO to expand a HadoopCatalog-specific AuthenticationParameters subclass. One way would be to express similar delegation settings as StorageConfigInfo
, or alternatively, HadoopCatalog is a special-case where the real storage credential is the catalog credential, so maybe we could just have something that "points" at the Catalog's StorageConfigInfo as the effective AuthenticationParameters.
@@ -862,6 +862,7 @@ components: | |||
type: string | |||
enum: | |||
- ICEBERG_REST | |||
- HADOOP |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since the HadoopCatalog
is highly unsafe if the set of people who have CATALOG_CREATE
privileges isn't exactly equal to the set of people with superuser privileges to run the entire Catalog service itself, before we introduce this one even as an option we should probably add a ALLOWED_CATALOG_CONNECTION_TYPES
or SUPPORTED_CATALOG_CONNECTION_TYPES
analogous to SUPPORTED_CATALOG_STORAGE_TYPES
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great idea, added
integration-tests/src/main/java/org/apache/polaris/service/it/env/ManagementApi.java
Outdated
Show resolved
Hide resolved
|
||
public HadoopConnectionConfigInfoDpo( | ||
@JsonProperty(value = "uri", required = true) @Nonnull String uri, | ||
@JsonProperty(value = "authenticationParameters", required = true) @Nonnull |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Right, identity-passthrough/delegation is important long-term, but has a lot of complicated prerequisites and other dependencies on how we express federated identities and user-isolated caching of metadata or storage credentials, so in the meantime we should just clearly document what the "powerful connectionconfig" model implies.
PolarisConfiguration.<List<String>>builder() | ||
.key("SUPPORTED_CATALOG_CONNECTION_TYPES") | ||
.description("The list of supported catalog connection types for federation") | ||
.defaultValue(List.of(ConnectionType.ICEBERG_REST.name(), ConnectionType.HADOOP.name())) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it possible to make this default an empty list, so that whoever wants to enable the federation feature has to explicitly opt-in to the connection types they want?
And/or set it to an empty list in application.properties. Or if empty list doesn't work for some reason, it seems ICEBERG_REST-only is a reasonable initial default since it's the minimal-dependency one long-term with a more well-defined security boundary.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's possible and I agree with the sentiment, but I would point to the default inclusion of FILE as a supported storage type as an important precedent here. Following this, it seems like what we should do is call out SUPPORTED_CATALOG_CONNECTION_TYPES
in configuring-polaris-for-production
. WDYT?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe one way we can stay consistent with this precedent but also be really safe-by-default (beyond what we do for storage types even) is to add a second flag gating off support for federation altogether... something like CATALOG_FEDERATION_ENABLED
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, we already have a feature gate for this: "ENABLE_CATALOG_FEDERATION"
However, at the moment enabling that only risks "broken experimental functionality" but not the security of the server, so the main concern would be if any service providers happen to already be setting ENABLE_CATALOG_FEDERATION=true
but aren't paying attention to the addition of new federation types.
Perhaps we can at least include the new SUPPORTED_CATALOG_CONNECTION_TYPES
in application.properties to explicitly list the two types, and maybe with a commented-out version that only includes ICEBERG_REST as well, with some comments about how the reader should choose the configuration. Or make the default one only [ICEBERG_REST]
with a commented-out version that includes both [ICEBERG_REST, HADOOP]
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Doesn't storage type FILE also risk the security of the server? It seems like almost the same situation there.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For now I'll update both the FeatureConfiguration default and the application.properties entry to only include ICEBERG_REST; when we add other federation types we can make a case-by-case evaluation of whether we want to enable those types by default.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
Polaris recently added support for catalog federation in #1305, which allows a Polaris instance to federate requests against a given catalog to another Iceberg REST catalog This is useful for deployments with multiple Iceberg REST catalogs, but we need to introduce federation to other catalogs in order to support a more diverge range of deployments.
This implements federation to a
Hadoop
catalog via the newHADOOP
type ofConnectionConfigInfo
.