-
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
Changes from all commits
9fb3c36
2608979
7d995ae
e88b968
90cc006
cefc3ed
497eb0e
7ea31bc
3dc4f7b
10ed9c7
5746b99
76fda06
5067e8e
a7854d7
84b3eb3
66e6220
5924230
dcd309f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,88 @@ | ||
/* | ||
* 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. | ||
*/ | ||
package org.apache.polaris.core.connection.hadoop; | ||
|
||
import com.fasterxml.jackson.annotation.JsonProperty; | ||
import com.google.common.base.MoreObjects; | ||
import jakarta.annotation.Nonnull; | ||
import jakarta.annotation.Nullable; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import org.apache.iceberg.CatalogProperties; | ||
import org.apache.polaris.core.admin.model.ConnectionConfigInfo; | ||
import org.apache.polaris.core.admin.model.HadoopConnectionConfigInfo; | ||
import org.apache.polaris.core.connection.AuthenticationParametersDpo; | ||
import org.apache.polaris.core.connection.ConnectionConfigInfoDpo; | ||
import org.apache.polaris.core.connection.ConnectionType; | ||
import org.apache.polaris.core.secrets.UserSecretsManager; | ||
|
||
/** | ||
* The internal persistence-object counterpart to {@link | ||
* org.apache.polaris.core.admin.model.HadoopConnectionConfigInfo} defined in the API model. | ||
*/ | ||
public class HadoopConnectionConfigInfoDpo extends ConnectionConfigInfoDpo { | ||
|
||
private final String warehouse; | ||
|
||
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 commentThe reason will be displayed to describe this comment to others. Learn more. QQ: What type of authentication method does Should we add a no-op authentication method? There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more. If we extend or refactor 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. 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I think the properties we pass into There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 commentThe 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 commentThe reason will be displayed to describe this comment to others. Learn more.
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 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) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
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 commentThe 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. |
||
AuthenticationParametersDpo authenticationParameters, | ||
@JsonProperty(value = "warehouse", required = false) @Nullable String remoteCatalogName) { | ||
super(ConnectionType.HADOOP.getCode(), uri, authenticationParameters); | ||
this.warehouse = remoteCatalogName; | ||
} | ||
|
||
public String getWarehouse() { | ||
return warehouse; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return MoreObjects.toStringHelper(this) | ||
.add("connectionTypeCode", getConnectionTypeCode()) | ||
.add("uri", getUri()) | ||
.add("warehouse", getWarehouse()) | ||
.add("authenticationParameters", getAuthenticationParameters().toString()) | ||
.toString(); | ||
} | ||
|
||
@Override | ||
public @Nonnull Map<String, String> asIcebergCatalogProperties( | ||
UserSecretsManager secretsManager) { | ||
HashMap<String, String> properties = new HashMap<>(); | ||
properties.put(CatalogProperties.URI, getUri()); | ||
if (getWarehouse() != null) { | ||
properties.put(CatalogProperties.WAREHOUSE_LOCATION, getWarehouse()); | ||
} | ||
properties.putAll(getAuthenticationParameters().asIcebergCatalogProperties(secretsManager)); | ||
return properties; | ||
} | ||
|
||
@Override | ||
public ConnectionConfigInfo asConnectionConfigInfoModel() { | ||
return HadoopConnectionConfigInfo.builder() | ||
.setConnectionType(ConnectionConfigInfo.ConnectionTypeEnum.HADOOP) | ||
.setUri(getUri()) | ||
.setWarehouse(getWarehouse()) | ||
.setAuthenticationParameters( | ||
getAuthenticationParameters().asAuthenticationParametersModel()) | ||
.build(); | ||
} | ||
} |
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-IcebergCatalog
?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