From 9907a95966c88188b1f91ed5fece85f495ace979 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Tue, 1 Apr 2025 21:45:59 -0700 Subject: [PATCH 01/29] add client change --- plugins/spark/v3.5/build.gradle.kts | 19 +++++-- .../polaris/spark/PolarisRESTCatalog.java | 51 +++++++++++++++++++ .../polaris/spark/PolarisSparkCatalog.java | 26 ++++++++++ .../rest/CreateGenericTableRESTRequest.java | 38 ++++++++++++++ .../rest/LoadGenericTableRESTResponse.java | 36 +++++++++++++ .../polaris/spark/utils/CatalogUtils.java | 0 .../polaris/core/rest/PolarisEndpoint.java | 32 ++++++++++++ 7 files changed, 198 insertions(+), 4 deletions(-) create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java create mode 100644 polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 36ca6d5285..de6123b44a 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -43,15 +43,26 @@ val spark35Version = pluginlibs.versions.spark35.get() dependencies { implementation(project(":polaris-api-iceberg-service")) { - // exclude the iceberg and jackson dependencies, use the - // dependencies packed in the iceberg-spark dependency + // exclude the iceberg dependencies, use the ones pulled + // by iceberg-core exclude("org.apache.iceberg", "*") - exclude("com.fasterxml.jackson.core", "*") } + implementation(project(":polaris-api-catalog-service")) + implementation(project(":polaris-core")) { exclude("org.apache.iceberg", "*") } implementation( "org.apache.iceberg:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}:${icebergVersion}" - ) + ) { + // exclude the iceberg rest dependencies, use the ones pulled + // with iceberg-core dependency + exclude("org.apache.iceberg.rest", "*") + } + + implementation("org.apache.iceberg:iceberg-core:${icebergVersion}") + + implementation("com.fasterxml.jackson.core:jackson-annotations") + implementation("com.fasterxml.jackson.core:jackson-core") + implementation("com.fasterxml.jackson.core:jackson-databind") compileOnly("org.apache.spark:spark-sql_${scalaVersion}:${spark35Version}") { // exclude log4j dependencies diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java new file mode 100644 index 0000000000..4c6eddbb6d --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -0,0 +1,51 @@ +/* + * 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.spark; + +import com.google.common.collect.ImmutableSet; +import org.apache.iceberg.io.CloseableGroup; +import org.apache.iceberg.rest.Endpoint; +import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.polaris.core.rest.PolarisEndpoint; +import org.apache.polaris.core.rest.PolarisResourcePaths; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.Set; +import java.util.logging.Logger; + +public class PolarisRESTCatalog implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(PolarisRESTCatalog.class); + + private RESTClient restClient = null; + private CloseableGroup closeables = null; + private Set endpoints; + private OAuth2Util.AuthSession catalogAuth = null; + private PolarisResourcePaths paths = null; + + private static final Set DEFAULT_ENDPOINTS = + ImmutableSet.builder() + .add(PolarisEndpoint.V1_CREATE_GENERIC_ABLE) + .add(PolarisEndpoint.V1_LOAD_GENERIC_TABLE) + .add(Endpoint.V1_LIST_TABLES) + .add(Endpoint.V1_DELETE_TABLE) + .build(); +} \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java new file mode 100644 index 0000000000..98c72e9650 --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -0,0 +1,26 @@ +/* + * 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.spark; + +import org.apache.spark.sql.connector.catalog.TableCatalog; + +public class PolarisSparkCatalog implements TableCatalog { + +} \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java new file mode 100644 index 0000000000..07d5701898 --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java @@ -0,0 +1,38 @@ +/* + * 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.spark.rest; + +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.iceberg.rest.RESTRequest; +import org.apache.polaris.service.types.CreateGenericTableRequest; + +public class CreateGenericTableRESTRequest extends CreateGenericTableRequest + implements RESTRequest { + + @JsonCreator + public CreateGenericTableRESTRequest(@JsonProperty(value = "name", required = true) String name, @JsonProperty(value = "format", required = true) String format, @JsonProperty(value = "doc") String doc, @JsonProperty(value = "properties") Map properties) { + super(name, format, doc, properties); + } + + @Override + public void validate() {} +} \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java new file mode 100644 index 0000000000..7a551419e9 --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java @@ -0,0 +1,36 @@ +/* + * 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.spark.rest; + +import org.apache.iceberg.rest.RESTResponse; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.polaris.service.types.GenericTable; +import org.apache.polaris.service.types.LoadGenericTableResponse; + +public class LoadGenericTableRESTResponse extends LoadGenericTableResponse implements RESTResponse { + + @JsonCreator + public LoadGenericTableRESTResponse(@JsonProperty(value = "table", required = true) GenericTable table) { + super(table); + } + + @Override + public void validate() {} +} \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java new file mode 100644 index 0000000000..e69de29bb2 diff --git a/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java b/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java new file mode 100644 index 0000000000..120fbf1604 --- /dev/null +++ b/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java @@ -0,0 +1,32 @@ +/* + * 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.rest; + +import org.apache.iceberg.rest.Endpoint; + +public class PolarisEndpoint { + public static final Endpoint V1_LIST_GENERIC_TABLES = + Endpoint.create("GET", PolarisResourcePaths.V1_GENERIC_TABLES); + public static final Endpoint V1_LOAD_GENERIC_TABLE = + Endpoint.create("GET", PolarisResourcePaths.V1_GENERIC_TABLE); + public static final Endpoint V1_CREATE_GENERIC_TABLE = + Endpoint.create("POST", PolarisResourcePaths.V1_GENERIC_TABLES); + public static final Endpoint V1_DELETE_GENERIC_TABLE = + Endpoint.create("DELETE", PolarisResourcePaths.V1_GENERIC_TABLE); +} \ No newline at end of file From 8fa79854a19fa240d3266134cec8102dc818bb8d Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 10:51:45 -0700 Subject: [PATCH 02/29] add rest catalog --- .../polaris/spark/PolarisRESTCatalog.java | 112 +++++++++++++++++- .../polaris/spark/PolarisSparkCatalog.java | 64 ++++++++++ .../apache/polaris/spark/SparkCatalog.java | 16 ++- .../polaris/spark/utils/CatalogUtils.java | 40 +++++++ 4 files changed, 227 insertions(+), 5 deletions(-) diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index 4c6eddbb6d..eaab63a6df 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -19,33 +19,137 @@ package org.apache.polaris.spark; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.CloseableGroup; -import org.apache.iceberg.rest.Endpoint; -import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.*; import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.iceberg.rest.responses.ConfigResponse; +import org.apache.iceberg.util.EnvironmentUtil; +import org.apache.iceberg.util.PropertyUtil; import org.apache.polaris.core.rest.PolarisEndpoint; import org.apache.polaris.core.rest.PolarisResourcePaths; +import org.apache.polaris.service.types.GenericTable; +import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.logging.Logger; public class PolarisRESTCatalog implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(PolarisRESTCatalog.class); + public static final String REST_PAGE_SIZE = "rest-page-size"; + private RESTClient restClient = null; private CloseableGroup closeables = null; private Set endpoints; private OAuth2Util.AuthSession catalogAuth = null; private PolarisResourcePaths paths = null; + private Integer pageSize = null; private static final Set DEFAULT_ENDPOINTS = ImmutableSet.builder() - .add(PolarisEndpoint.V1_CREATE_GENERIC_ABLE) + .add(PolarisEndpoint.V1_CREATE_GENERIC_TABLE) .add(PolarisEndpoint.V1_LOAD_GENERIC_TABLE) .add(Endpoint.V1_LIST_TABLES) .add(Endpoint.V1_DELETE_TABLE) .build(); + + public void initialize(Map unresolved, OAuth2Util.AuthSession catalogAuth) { + Preconditions.checkArgument(unresolved != null, "Invalid configuration: null"); + + // resolve any configuration that is supplied by environment variables + Map props = EnvironmentUtil.resolveAll(unresolved); + + // TODO: switch to use authManager once iceberg dependency is updated to 1.9.0 + this.catalogAuth = catalogAuth; + + this.restClient = + HTTPClient.builder(props) + .uri(props.get(CatalogProperties.URI)) + .build() + .withAuthSession(catalogAuth); + + // call getConfig to get the server configurations + ConfigResponse config = fetchConfig(this.restClient, catalogAuth.headers(), props); + Map mergedProps = config.merge(props); + if (config.endpoints().isEmpty()) { + this.endpoints = DEFAULT_ENDPOINTS; + } else { + this.endpoints = ImmutableSet.copyOf(config.endpoints()); + } + + this.paths = PolarisResourcePaths.forCatalogProperties(mergedProps); + this.restClient = + HTTPClient.builder(mergedProps) + .uri(mergedProps.get(CatalogProperties.URI)) + .build() + .withAuthSession(catalogAuth); + + this.pageSize = PropertyUtil.propertyAsNullableInt(mergedProps, REST_PAGE_SIZE); + if (pageSize != null) { + Preconditions.checkArgument( + pageSize > 0, "Invalid value for %s, must be a positive integer", REST_PAGE_SIZE); + } + + this.closeables = new CloseableGroup(); + this.closeables.addCloseable(this.restClient); + this.closeables.setSuppressCloseFailure(true); + + } + + protected static ConfigResponse fetchConfig( + RESTClient client, Map headers, Map properties) { + // send the client's warehouse location to the service to keep in sync + // this is needed for cases where the warehouse is configured at client side, + // and used by Polaris server as catalog name. + ImmutableMap.Builder queryParams = ImmutableMap.builder(); + if (properties.containsKey(CatalogProperties.WAREHOUSE_LOCATION)) { + queryParams.put( + CatalogProperties.WAREHOUSE_LOCATION, + properties.get(CatalogProperties.WAREHOUSE_LOCATION)); + } + + ConfigResponse configResponse = + client.get( + ResourcePaths.config(), + queryParams.build(), + ConfigResponse.class, + headers, + ErrorHandlers.defaultErrorHandler()); + configResponse.validate(); + return configResponse; + } + + @Override + public void close() throws IOException { + if (closeables != null) { + closeables.close(); + } + } + + public List listTables(Namespace ns) { + throw new UnsupportedOperationException("listTables not supported"); + } + + public boolean dropTable(TableIdentifier identifier) { + throw new UnsupportedOperationException("dropTable not supported"); + } + + public GenericTable createTable(TableIdentifier identifier, String format, Map props) { + throw new UnsupportedOperationException("CreateTable not supported"); + } + + public GenericTable loadTable(TableIdentifier identifier) { + throw new UnsupportedOperationException("loadTable not supported"); + } + } \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java index 98c72e9650..3e0c2c9453 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -19,8 +19,72 @@ package org.apache.polaris.spark; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; public class PolarisSparkCatalog implements TableCatalog { + private static final Logger LOG = LoggerFactory.getLogger(PolarisSparkCatalog.class); + + private PolarisRESTCatalog restCatalog = null; + private String catalogName = null; + + public PolarisSparkCatalog(PolarisRESTCatalog restCatalog) { + this.restCatalog = restCatalog; + } + + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + this.catalogName = name; + } + + @Override + public String name() { + return catalogName; + } + + @Override + public Table loadTable(Identifier identifier) throws NoSuchTableException { + throw new UnsupportedOperationException("load table is not supported"); + } + + @Override + public Table createTable( + Identifier identifier, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { + throw new UnsupportedOperationException("create table is not supported"); + } + + @Override + public Table alterTable(Identifier identifier, TableChange... changes) throws NoSuchTableException { + throw new NoSuchTableException(identifier); + } + + @Override + public boolean dropTable(Identifier identifier) { + return false; + } + + @Override + public void renameTable(Identifier from, Identifier to) + throws NoSuchTableException, TableAlreadyExistsException { + throw new UnsupportedOperationException("renameTable operation is not supported"); + } + + @Override + public Identifier[] listTables(String[] namespace) { + throw new UnsupportedOperationException("listTables operation is not supported"); + } + } \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index e38bbe1ad0..e6f30bb39c 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -39,6 +39,10 @@ import org.apache.spark.sql.connector.catalog.View; import org.apache.spark.sql.connector.catalog.ViewCatalog; import org.apache.spark.sql.connector.catalog.ViewChange; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -53,8 +57,8 @@ public class SparkCatalog private static final Set DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER); private String catalogName = null; private org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; + private PolarisSparkCatalog polarisSparkCatalog = null; - // TODO: Add Polaris Specific REST Catalog @Override public String name() { @@ -63,6 +67,16 @@ public String name() { @Override public void initialize(String name, CaseInsensitiveStringMap options) { + String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); + if (catalogImpl != null) { + throw new UnsupportedOperationException("Customized catalog implementation is currently not supported!"); + } + String catalogType = + PropertyUtil.propertyAsString(options, CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_REST); + if (!catalogType.equals(CatalogUtil.ICEBERG_CATALOG_REST)) { + throw new UnsupportedOperationException("Only rest catalog type is supported, but got catalog type: " + catalogType); + } + this.catalogName = name; this.icebergsSparkCatalog = new org.apache.iceberg.spark.SparkCatalog(); this.icebergsSparkCatalog.initialize(name, options); diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java index e69de29bb2..ad31f94d4e 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java @@ -0,0 +1,40 @@ +/* + * 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.spark.utils; + +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; + +public class CatalogUtils { + public static void checkNamespaceIsValid(Namespace namespace) { + if (namespace.isEmpty()) { + throw new NoSuchNamespaceException("Invalid namespace: %s", namespace); + } + } + + public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { + if (tableIdentifier.namespace().isEmpty()) { + throw new NoSuchTableException("Invalid table identifier: %s", tableIdentifier); + } + } + +} \ No newline at end of file From 96cc40b526ef7081c49357b01f85df00c6b658b1 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 13:04:57 -0700 Subject: [PATCH 03/29] add helper --- .../polaris/spark/PolarisRESTCatalog.java | 5 +- .../apache/polaris/spark/SparkCatalog.java | 4 + .../polaris/spark/utils/CatalogUtils.java | 31 +++++++ .../polaris/spark/utils/DeltaHelper.java | 81 +++++++++++++++++++ 4 files changed, 119 insertions(+), 2 deletions(-) create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index eaab63a6df..a7cc7db5cb 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -55,12 +55,13 @@ public class PolarisRESTCatalog implements Closeable { private PolarisResourcePaths paths = null; private Integer pageSize = null; + // TODO: update to use the predefined GENERIC_TABLE_ENDPOINTS private static final Set DEFAULT_ENDPOINTS = ImmutableSet.builder() .add(PolarisEndpoint.V1_CREATE_GENERIC_TABLE) .add(PolarisEndpoint.V1_LOAD_GENERIC_TABLE) - .add(Endpoint.V1_LIST_TABLES) - .add(Endpoint.V1_DELETE_TABLE) + .add(PolarisEndpoint.V1_DELETE_GENERIC_TABLE) + .add(PolarisEndpoint.V1_LIST_GENERIC_TABLES) .build(); public void initialize(Map unresolved, OAuth2Util.AuthSession catalogAuth) { diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index e6f30bb39c..f732406a56 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -46,6 +46,8 @@ import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SparkCatalog implements StagingTableCatalog, @@ -53,8 +55,10 @@ public class SparkCatalog SupportsNamespaces, ViewCatalog, SupportsReplaceView { + private static final Logger LOG = LoggerFactory.getLogger(SparkCatalog.class); private static final Set DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER); + private String catalogName = null; private org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; private PolarisSparkCatalog polarisSparkCatalog = null; diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java index ad31f94d4e..ecb33d8e60 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java @@ -23,6 +23,12 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.rest.RESTSessionCatalog; +import org.apache.iceberg.rest.auth.OAuth2Util; +import org.apache.iceberg.spark.SparkCatalog; + +import java.lang.reflect.Field; public class CatalogUtils { public static void checkNamespaceIsValid(Namespace namespace) { @@ -37,4 +43,29 @@ public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { } } + /** + * Get the catalogAuth field inside the RESTSessionCatalog used by Iceberg + * Spark Catalog use reflection. + * TODO: Deprecate this function once the iceberg client is updated to 1.9.0 to use + * AuthManager and the capability of injecting an AuthManger is available. + * Related iceberg PR: https://github.com/apache/iceberg/pull/12655 + */ + public static OAuth2Util.AuthSession getAuthSession(SparkCatalog sparkCatalog) { + try { + Field icebergRestCatalogField = sparkCatalog.getClass().getDeclaredField("icebergCatalog"); + icebergRestCatalogField.setAccessible(true); + RESTCatalog icebergRestCatalog = (RESTCatalog) icebergRestCatalogField.get(sparkCatalog); + + Field sessionCatalogField = icebergRestCatalog.getClass().getDeclaredField("sessionCatalog"); + sessionCatalogField.setAccessible(true); + RESTSessionCatalog sessionCatalog = + (RESTSessionCatalog) sessionCatalogField.get(icebergRestCatalog); + + Field authField = sessionCatalog.getClass().getDeclaredField("catalogAuth"); + authField.setAccessible(true); + return (OAuth2Util.AuthSession) authField.get(sessionCatalog); + } catch (Exception e) { + throw new RuntimeException("Failed to get the catalogAuth from the Iceberg SparkCatalog", e); + } + } } \ No newline at end of file diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java new file mode 100644 index 0000000000..43eb5ad82f --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -0,0 +1,81 @@ +/* + * 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.spark.utils; + +import org.apache.iceberg.common.DynConstructors; +import org.apache.polaris.spark.PolarisSparkCatalog; +import org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DeltaHelper { + private static final Logger LOG = LoggerFactory.getLogger(DeltaHelper.class); + + private static final String DELTA_CATALOG_CLASS = "org.apache.spark.sql.delta.catalog.DeltaCatalog"; + private TableCatalog deltaCatalog = null; + + public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { + if (this.deltaCatalog != null) { + return this.deltaCatalog; + } + + DynConstructors.Ctor ctor; + try { + ctor = DynConstructors.builder(TableCatalog.class).impl(DELTA_CATALOG_CLASS).buildChecked(); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException( + String.format( + "Cannot initialize Delta Catalog %s: %s", DELTA_CATALOG_CLASS, e.getMessage()), + e); + } + + try { + this.deltaCatalog = ctor.newInstance(); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + String.format("Cannot initialize Delta Catalog, %s does not implement Table Catalog.", DELTA_CATALOG_CLASS), + e); + } + + ((DelegatingCatalogExtension) this.deltaCatalog).setDelegateCatalog(g); + + // https://github.com/delta-io/delta/issues/4306 + if (this.sparkTableCatalog instanceof DeltaCatalog) { + try { + Field field = sparkTableCatalog.getClass().getDeclaredField("isUnityCatalog"); + field.setAccessible(true); + // Access the lazy val field's underlying method (generated by Scala) + String methodGetName = "isUnityCatalog" + "$lzycompute"; + Method method = sparkTableCatalog.getClass().getDeclaredMethod(methodGetName); + method.setAccessible(true); + method.invoke(sparkTableCatalog); + // sparkTableCatalog.getClass().getMethod("isUnityCatalog").invoke(sparkTableCatalog); // + // force init in case it's a lazy val + // LOG.warn("the isUnityCatalog have value before {}", field.get(sparkTableCatalog) ); + field.set(sparkTableCatalog, true); + LOG.warn("the isUnityCatalog have value {}", field.get(sparkTableCatalog)); + } catch (Exception e) { + throw new RuntimeException("Failed to set the isUnityCatalog field", e); + } + } + return this.deltaCatalog; + } +} \ No newline at end of file From 6ffa5bb42ef7f770ee27e5dc93dd095c70b7f70e Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 13:25:09 -0700 Subject: [PATCH 04/29] add change --- .../polaris/spark/utils/DeltaHelper.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java index 43eb5ad82f..5c7a67b667 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -26,6 +26,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.Field; +import java.lang.reflect.Method; + public class DeltaHelper { private static final Logger LOG = LoggerFactory.getLogger(DeltaHelper.class); @@ -55,16 +58,16 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { e); } - ((DelegatingCatalogExtension) this.deltaCatalog).setDelegateCatalog(g); + // set the polaris spark catalog as the delegate catalog of delta catalog + ((DelegatingCatalogExtension) this.deltaCatalog).setDelegateCatalog(polarisSparkCatalog); // https://github.com/delta-io/delta/issues/4306 - if (this.sparkTableCatalog instanceof DeltaCatalog) { - try { - Field field = sparkTableCatalog.getClass().getDeclaredField("isUnityCatalog"); + try { + Field field = this.deltaCatalog.getClass().getDeclaredField("isUnityCatalog"); field.setAccessible(true); // Access the lazy val field's underlying method (generated by Scala) String methodGetName = "isUnityCatalog" + "$lzycompute"; - Method method = sparkTableCatalog.getClass().getDeclaredMethod(methodGetName); + Method method = this.deltaCatalog.getClass().getDeclaredMethod(methodGetName); method.setAccessible(true); method.invoke(sparkTableCatalog); // sparkTableCatalog.getClass().getMethod("isUnityCatalog").invoke(sparkTableCatalog); // @@ -72,9 +75,10 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { // LOG.warn("the isUnityCatalog have value before {}", field.get(sparkTableCatalog) ); field.set(sparkTableCatalog, true); LOG.warn("the isUnityCatalog have value {}", field.get(sparkTableCatalog)); - } catch (Exception e) { + } catch (NoSuchFieldException e) { throw new RuntimeException("Failed to set the isUnityCatalog field", e); - } + } catch (NoSuchMethodException e) { + throw new } return this.deltaCatalog; } From 1d0155172fe47c9d724ed025db332e552bd75f9f Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 14:57:24 -0700 Subject: [PATCH 05/29] add change --- .../apache/polaris/spark/utils/DeltaHelper.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java index 5c7a67b667..12c58e8353 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -65,20 +65,17 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { try { Field field = this.deltaCatalog.getClass().getDeclaredField("isUnityCatalog"); field.setAccessible(true); - // Access the lazy val field's underlying method (generated by Scala) + // Access the lazy val field's underlying method String methodGetName = "isUnityCatalog" + "$lzycompute"; Method method = this.deltaCatalog.getClass().getDeclaredMethod(methodGetName); method.setAccessible(true); - method.invoke(sparkTableCatalog); - // sparkTableCatalog.getClass().getMethod("isUnityCatalog").invoke(sparkTableCatalog); // - // force init in case it's a lazy val - // LOG.warn("the isUnityCatalog have value before {}", field.get(sparkTableCatalog) ); - field.set(sparkTableCatalog, true); - LOG.warn("the isUnityCatalog have value {}", field.get(sparkTableCatalog)); + // invoke the lazy methods before it + method.invoke(this.deltaCatalog); + field.set(this.deltaCatalog, true); } catch (NoSuchFieldException e) { - throw new RuntimeException("Failed to set the isUnityCatalog field", e); + throw new RuntimeException("Failed find the isUnityCatalog field, delt", e); } catch (NoSuchMethodException e) { - throw new + throw new RuntimeException("Failed to invoke isUnityCatalog evaluation method", e); } return this.deltaCatalog; } From 32ee7157c2c6cea1d9d3b78c42419e38702da75b Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 17:51:49 -0700 Subject: [PATCH 06/29] fix error --- plugins/spark/v3.5/build.gradle.kts | 2 +- .../polaris/spark/PolarisRESTCatalog.java | 62 +++++++++++++------ .../polaris/spark/PolarisSparkCatalog.java | 30 ++++++--- .../apache/polaris/spark/SparkCatalog.java | 49 +++++++++++---- .../rest/CreateGenericTableRESTRequest.java | 11 ++-- .../rest/LoadGenericTableRESTResponse.java | 7 ++- .../polaris/spark/utils/DeltaHelper.java | 48 ++++++++------ ...logUtils.java => PolarisCatalogUtils.java} | 59 +++++++++++++++--- .../polaris/core/rest/PolarisEndpoint.java | 32 ---------- 9 files changed, 194 insertions(+), 106 deletions(-) rename plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/{CatalogUtils.java => PolarisCatalogUtils.java} (53%) delete mode 100644 polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index de6123b44a..b28072b034 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -106,7 +106,7 @@ tasks.register("createPolarisSparkJar") { // Optimization: Minimize the JAR (remove unused classes from dependencies) // The iceberg-spark-runtime plugin is always packaged along with our polaris-spark plugin, // therefore excluded from the optimization. - minimize { exclude(dependency("org.apache.iceberg:iceberg-spark-runtime-*.*")) } + // minimize { exclude(dependency("org.apache.iceberg:iceberg-spark-runtime-*.*")) } } tasks.withType(Jar::class).named("sourcesJar") { dependsOn("createPolarisSparkJar") } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index a7cc7db5cb..9fa076d846 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -16,11 +16,15 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.polaris.spark; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -31,18 +35,15 @@ import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.util.EnvironmentUtil; import org.apache.iceberg.util.PropertyUtil; -import org.apache.polaris.core.rest.PolarisEndpoint; +import org.apache.polaris.core.rest.PolarisEndpoints; import org.apache.polaris.core.rest.PolarisResourcePaths; import org.apache.polaris.service.types.GenericTable; +import org.apache.polaris.spark.rest.CreateGenericTableRESTRequest; +import org.apache.polaris.spark.rest.LoadGenericTableRESTResponse; +import org.apache.polaris.spark.utils.PolarisCatalogUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Set; - public class PolarisRESTCatalog implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(PolarisRESTCatalog.class); @@ -58,10 +59,10 @@ public class PolarisRESTCatalog implements Closeable { // TODO: update to use the predefined GENERIC_TABLE_ENDPOINTS private static final Set DEFAULT_ENDPOINTS = ImmutableSet.builder() - .add(PolarisEndpoint.V1_CREATE_GENERIC_TABLE) - .add(PolarisEndpoint.V1_LOAD_GENERIC_TABLE) - .add(PolarisEndpoint.V1_DELETE_GENERIC_TABLE) - .add(PolarisEndpoint.V1_LIST_GENERIC_TABLES) + .add(PolarisEndpoints.V1_CREATE_GENERIC_TABLE) + .add(PolarisEndpoints.V1_LOAD_GENERIC_TABLE) + .add(PolarisEndpoints.V1_DELETE_GENERIC_TABLE) + .add(PolarisEndpoints.V1_LIST_GENERIC_TABLES) .build(); public void initialize(Map unresolved, OAuth2Util.AuthSession catalogAuth) { @@ -104,7 +105,6 @@ public void initialize(Map unresolved, OAuth2Util.AuthSession ca this.closeables = new CloseableGroup(); this.closeables.addCloseable(this.restClient); this.closeables.setSuppressCloseFailure(true); - } protected static ConfigResponse fetchConfig( @@ -145,12 +145,38 @@ public boolean dropTable(TableIdentifier identifier) { throw new UnsupportedOperationException("dropTable not supported"); } - public GenericTable createTable(TableIdentifier identifier, String format, Map props) { - throw new UnsupportedOperationException("CreateTable not supported"); + public GenericTable createTable( + TableIdentifier identifier, String format, Map props) { + Endpoint.check(endpoints, PolarisEndpoints.V1_CREATE_GENERIC_TABLE); + CreateGenericTableRESTRequest request = + new CreateGenericTableRESTRequest(identifier.name(), format, null, props); + + LoadGenericTableRESTResponse response = + restClient + .withAuthSession(this.catalogAuth) + .post( + paths.genericTables(identifier.namespace()), + request, + LoadGenericTableRESTResponse.class, + Map.of(), + ErrorHandlers.tableErrorHandler()); + + return response.getTable(); } public GenericTable loadTable(TableIdentifier identifier) { - throw new UnsupportedOperationException("loadTable not supported"); + Endpoint.check(endpoints, PolarisEndpoints.V1_LOAD_GENERIC_TABLE); + PolarisCatalogUtils.checkIdentifierIsValid(identifier); + LoadGenericTableRESTResponse response = + restClient + .withAuthSession(this.catalogAuth) + .get( + paths.genericTable(identifier), + null, + LoadGenericTableRESTResponse.class, + Map.of(), + ErrorHandlers.tableErrorHandler()); + + return response.getTable(); } - -} \ No newline at end of file +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java index 3e0c2c9453..188a149fb5 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -16,9 +16,12 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.polaris.spark; +import java.util.Map; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.polaris.service.types.GenericTable; +import org.apache.polaris.spark.utils.PolarisCatalogUtils; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; @@ -32,8 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; - public class PolarisSparkCatalog implements TableCatalog { private static final Logger LOG = LoggerFactory.getLogger(PolarisSparkCatalog.class); @@ -56,17 +57,28 @@ public String name() { @Override public Table loadTable(Identifier identifier) throws NoSuchTableException { - throw new UnsupportedOperationException("load table is not supported"); + GenericTable genericTable = + this.restCatalog.loadTable(Spark3Util.identifierToTableIdentifier(identifier)); + return PolarisCatalogUtils.loadSparkTable(genericTable); } @Override public Table createTable( - Identifier identifier, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { - throw new UnsupportedOperationException("create table is not supported"); + Identifier identifier, + StructType schema, + Transform[] transforms, + Map properties) + throws TableAlreadyExistsException, NoSuchNamespaceException { + String format = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); + GenericTable genericTable = + this.restCatalog.createTable( + Spark3Util.identifierToTableIdentifier(identifier), format, properties); + return PolarisCatalogUtils.loadSparkTable(genericTable); } @Override - public Table alterTable(Identifier identifier, TableChange... changes) throws NoSuchTableException { + public Table alterTable(Identifier identifier, TableChange... changes) + throws NoSuchTableException { throw new NoSuchTableException(identifier); } @@ -85,6 +97,4 @@ public void renameTable(Identifier from, Identifier to) public Identifier[] listTables(String[] namespace) { throw new UnsupportedOperationException("listTables operation is not supported"); } - - -} \ No newline at end of file +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index f732406a56..4446fd419b 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -18,7 +18,6 @@ */ package org.apache.polaris.spark; -import com.google.common.collect.ImmutableSet; import java.util.Map; import java.util.Set; import org.apache.iceberg.spark.SupportsReplaceView; @@ -41,8 +40,10 @@ import org.apache.spark.sql.connector.catalog.ViewChange; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.util.PropertyUtil; +import org.apache.polaris.spark.utils.DeltaHelper; +import org.apache.polaris.spark.utils.PolarisCatalogUtils; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -57,12 +58,11 @@ public class SparkCatalog SupportsReplaceView { private static final Logger LOG = LoggerFactory.getLogger(SparkCatalog.class); - private static final Set DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER); - private String catalogName = null; private org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; private PolarisSparkCatalog polarisSparkCatalog = null; + private DeltaHelper deltaHelper = null; @Override public String name() { @@ -72,30 +72,57 @@ public String name() { @Override public void initialize(String name, CaseInsensitiveStringMap options) { String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); + // TODO: relax this once AuthManager reuse enabled if (catalogImpl != null) { - throw new UnsupportedOperationException("Customized catalog implementation is currently not supported!"); + throw new UnsupportedOperationException( + "Customized catalog implementation is currently not supported!"); } String catalogType = - PropertyUtil.propertyAsString(options, CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_REST); - if (!catalogType.equals(CatalogUtil.ICEBERG_CATALOG_REST)) { - throw new UnsupportedOperationException("Only rest catalog type is supported, but got catalog type: " + catalogType); + PropertyUtil.propertyAsString( + options, CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); + if (!catalogType.equals(CatalogUtil.ICEBERG_CATALOG_TYPE_REST)) { + throw new UnsupportedOperationException( + "Only rest catalog type is supported, but got catalog type: " + catalogType); } this.catalogName = name; + // initialize the icebergSparkCatalog this.icebergsSparkCatalog = new org.apache.iceberg.spark.SparkCatalog(); this.icebergsSparkCatalog.initialize(name, options); + + // initialize the polaris spark catalog + OAuth2Util.AuthSession catalogAuth = + PolarisCatalogUtils.getAuthSession(this.icebergsSparkCatalog); + PolarisRESTCatalog restCatalog = new PolarisRESTCatalog(); + restCatalog.initialize(options, catalogAuth); + this.polarisSparkCatalog = new PolarisSparkCatalog(restCatalog); + this.polarisSparkCatalog.initialize(name, options); + + this.deltaHelper = new DeltaHelper(); } @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - throw new UnsupportedOperationException("loadTable"); + try { + return this.icebergsSparkCatalog.loadTable(ident); + } catch (NoSuchTableException e) { + return this.polarisSparkCatalog.loadTable(ident); + } } @Override public Table createTable( Identifier ident, StructType schema, Transform[] transforms, Map properties) - throws TableAlreadyExistsException { - throw new UnsupportedOperationException("createTable"); + throws TableAlreadyExistsException, NoSuchNamespaceException { + String provider = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); + if (PolarisCatalogUtils.useIceberg(provider)) { + return this.icebergsSparkCatalog.createTable(ident, schema, transforms, properties); + } else if (PolarisCatalogUtils.useDelta(provider)) { + TableCatalog deltaCatalog = deltaHelper.loadDeltaCatalog(this.polarisSparkCatalog); + return deltaCatalog.createTable(ident, schema, transforms, properties); + } else { + return this.polarisSparkCatalog.createTable(ident, schema, transforms, properties); + } } @Override diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java index 07d5701898..ce0a5c17af 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java @@ -18,10 +18,9 @@ */ package org.apache.polaris.spark.rest; -import java.util.Map; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; import org.apache.iceberg.rest.RESTRequest; import org.apache.polaris.service.types.CreateGenericTableRequest; @@ -29,10 +28,14 @@ public class CreateGenericTableRESTRequest extends CreateGenericTableRequest implements RESTRequest { @JsonCreator - public CreateGenericTableRESTRequest(@JsonProperty(value = "name", required = true) String name, @JsonProperty(value = "format", required = true) String format, @JsonProperty(value = "doc") String doc, @JsonProperty(value = "properties") Map properties) { + public CreateGenericTableRESTRequest( + @JsonProperty(value = "name", required = true) String name, + @JsonProperty(value = "format", required = true) String format, + @JsonProperty(value = "doc") String doc, + @JsonProperty(value = "properties") Map properties) { super(name, format, doc, properties); } @Override public void validate() {} -} \ No newline at end of file +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java index 7a551419e9..799942cb99 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java @@ -18,19 +18,20 @@ */ package org.apache.polaris.spark.rest; -import org.apache.iceberg.rest.RESTResponse; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.iceberg.rest.RESTResponse; import org.apache.polaris.service.types.GenericTable; import org.apache.polaris.service.types.LoadGenericTableResponse; public class LoadGenericTableRESTResponse extends LoadGenericTableResponse implements RESTResponse { @JsonCreator - public LoadGenericTableRESTResponse(@JsonProperty(value = "table", required = true) GenericTable table) { + public LoadGenericTableRESTResponse( + @JsonProperty(value = "table", required = true) GenericTable table) { super(table); } @Override public void validate() {} -} \ No newline at end of file +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java index 12c58e8353..48c3bd50d9 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -16,9 +16,10 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.polaris.spark.utils; +import java.lang.reflect.Field; +import java.lang.reflect.Method; import org.apache.iceberg.common.DynConstructors; import org.apache.polaris.spark.PolarisSparkCatalog; import org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension; @@ -26,13 +27,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.lang.reflect.Field; -import java.lang.reflect.Method; - public class DeltaHelper { private static final Logger LOG = LoggerFactory.getLogger(DeltaHelper.class); - private static final String DELTA_CATALOG_CLASS = "org.apache.spark.sql.delta.catalog.DeltaCatalog"; + private static final String DELTA_CATALOG_CLASS = + "org.apache.spark.sql.delta.catalog.DeltaCatalog"; private TableCatalog deltaCatalog = null; public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { @@ -54,7 +53,9 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { this.deltaCatalog = ctor.newInstance(); } catch (ClassCastException e) { throw new IllegalArgumentException( - String.format("Cannot initialize Delta Catalog, %s does not implement Table Catalog.", DELTA_CATALOG_CLASS), + String.format( + "Cannot initialize Delta Catalog, %s does not implement Table Catalog.", + DELTA_CATALOG_CLASS), e); } @@ -62,21 +63,32 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { ((DelegatingCatalogExtension) this.deltaCatalog).setDelegateCatalog(polarisSparkCatalog); // https://github.com/delta-io/delta/issues/4306 + try { + // Access the lazy val field's underlying method + String methodGetName = "isUnityCatalog" + "$lzycompute"; + Method method = this.deltaCatalog.getClass().getDeclaredMethod(methodGetName); + method.setAccessible(true); + // invoke the lazy methods before it is set + method.invoke(this.deltaCatalog); + } catch (NoSuchMethodException e) { + throw new RuntimeException( + "Failed to find lazy compute method for isUnityCatalog, delta-spark version >= 3.2.1 is required", + e); + } catch (Exception e) { + throw new RuntimeException("Failed to invoke the lazy compute methods for isUnityCatalog", e); + } + try { Field field = this.deltaCatalog.getClass().getDeclaredField("isUnityCatalog"); - field.setAccessible(true); - // Access the lazy val field's underlying method - String methodGetName = "isUnityCatalog" + "$lzycompute"; - Method method = this.deltaCatalog.getClass().getDeclaredMethod(methodGetName); - method.setAccessible(true); - // invoke the lazy methods before it - method.invoke(this.deltaCatalog); - field.set(this.deltaCatalog, true); + field.setAccessible(true); + field.set(this.deltaCatalog, true); } catch (NoSuchFieldException e) { - throw new RuntimeException("Failed find the isUnityCatalog field, delt", e); - } catch (NoSuchMethodException e) { - throw new RuntimeException("Failed to invoke isUnityCatalog evaluation method", e); + throw new RuntimeException( + "Failed find the isUnityCatalog field, delta-spark version >= 3.2.1 is required", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Failed to set the isUnityCatalog field", e); } + return this.deltaCatalog; } -} \ No newline at end of file +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java similarity index 53% rename from plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java rename to plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index ecb33d8e60..6d26587157 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/CatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -16,9 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.polaris.spark.utils; +import com.google.common.collect.Maps; +import java.lang.reflect.Field; +import java.util.Map; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchNamespaceException; @@ -27,10 +29,20 @@ import org.apache.iceberg.rest.RESTSessionCatalog; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.spark.SparkCatalog; +import org.apache.polaris.service.types.GenericTable; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.execution.datasources.DataSource; +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import java.lang.reflect.Field; +public class PolarisCatalogUtils { + public static final String TABLE_PROVIDER_KEY = "provider"; + public static final String TABLE_PATH_KEY = "path"; -public class CatalogUtils { public static void checkNamespaceIsValid(Namespace namespace) { if (namespace.isEmpty()) { throw new NoSuchNamespaceException("Invalid namespace: %s", namespace); @@ -43,12 +55,41 @@ public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { } } + public static boolean useIceberg(String provider) { + return provider == null || "iceberg".equalsIgnoreCase(provider); + } + + public static boolean useDelta(String provider) { + return "delta".equalsIgnoreCase(provider); + } + + public static Table loadSparkTable(GenericTable genericTable) { + SparkSession sparkSession = SparkSession.active(); + TableProvider provider = + DataSource.lookupDataSourceV2(genericTable.getFormat(), sparkSession.sessionState().conf()) + .get(); + Map properties = genericTable.getProperties(); + boolean hasLocationClause = + properties.containsKey(TableCatalog.PROP_LOCATION) + && (properties.get(TableCatalog.PROP_LOCATION) != null); + boolean hasPathClause = + properties.containsKey(TABLE_PATH_KEY) && (properties.get(TABLE_PATH_KEY) != null); + Map tableProperties = Maps.newHashMap(); + tableProperties.putAll(properties); + if (!hasPathClause && hasLocationClause) { + // DataSourceV2 requires the path for table loading + tableProperties.put(TABLE_PATH_KEY, properties.get(TableCatalog.PROP_LOCATION)); + } + CaseInsensitiveStringMap property_map = new CaseInsensitiveStringMap(tableProperties); + return DataSourceV2Utils.getTableFromProvider( + provider, property_map, scala.Option$.MODULE$.empty()); + } + /** - * Get the catalogAuth field inside the RESTSessionCatalog used by Iceberg - * Spark Catalog use reflection. - * TODO: Deprecate this function once the iceberg client is updated to 1.9.0 to use - * AuthManager and the capability of injecting an AuthManger is available. - * Related iceberg PR: https://github.com/apache/iceberg/pull/12655 + * Get the catalogAuth field inside the RESTSessionCatalog used by Iceberg Spark Catalog use + * reflection. TODO: Deprecate this function once the iceberg client is updated to 1.9.0 to use + * AuthManager and the capability of injecting an AuthManger is available. Related iceberg PR: + * https://github.com/apache/iceberg/pull/12655 */ public static OAuth2Util.AuthSession getAuthSession(SparkCatalog sparkCatalog) { try { @@ -68,4 +109,4 @@ public static OAuth2Util.AuthSession getAuthSession(SparkCatalog sparkCatalog) { throw new RuntimeException("Failed to get the catalogAuth from the Iceberg SparkCatalog", e); } } -} \ No newline at end of file +} diff --git a/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java b/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java deleted file mode 100644 index 120fbf1604..0000000000 --- a/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisEndpoint.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.rest; - -import org.apache.iceberg.rest.Endpoint; - -public class PolarisEndpoint { - public static final Endpoint V1_LIST_GENERIC_TABLES = - Endpoint.create("GET", PolarisResourcePaths.V1_GENERIC_TABLES); - public static final Endpoint V1_LOAD_GENERIC_TABLE = - Endpoint.create("GET", PolarisResourcePaths.V1_GENERIC_TABLE); - public static final Endpoint V1_CREATE_GENERIC_TABLE = - Endpoint.create("POST", PolarisResourcePaths.V1_GENERIC_TABLES); - public static final Endpoint V1_DELETE_GENERIC_TABLE = - Endpoint.create("DELETE", PolarisResourcePaths.V1_GENERIC_TABLE); -} \ No newline at end of file From 2d0111ce3265339172932b6c768543019bb44058 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 22:45:40 -0700 Subject: [PATCH 07/29] add test --- plugins/spark/v3.5/build.gradle.kts | 1 + .../polaris/spark/PolarisRESTCatalog.java | 12 ++++++------ .../polaris/spark/PolarisSparkCatalog.java | 4 ++-- .../spark/utils/PolarisCatalogUtils.java | 17 ++++++++++++++--- 4 files changed, 23 insertions(+), 11 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index b28072b034..a0b4ca0a71 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -56,6 +56,7 @@ dependencies { // exclude the iceberg rest dependencies, use the ones pulled // with iceberg-core dependency exclude("org.apache.iceberg.rest", "*") + exclude("org.apache.iceberg.hadoop", "*") } implementation("org.apache.iceberg:iceberg-core:${icebergVersion}") diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index 9fa076d846..b4f996a3c6 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -137,17 +137,17 @@ public void close() throws IOException { } } - public List listTables(Namespace ns) { + public List listGenericTables(Namespace ns) { throw new UnsupportedOperationException("listTables not supported"); } - public boolean dropTable(TableIdentifier identifier) { + public boolean dropGenericTable(TableIdentifier identifier) { throw new UnsupportedOperationException("dropTable not supported"); } - public GenericTable createTable( + public GenericTable createGenericTable( TableIdentifier identifier, String format, Map props) { - Endpoint.check(endpoints, PolarisEndpoints.V1_CREATE_GENERIC_TABLE); + // Endpoint.check(endpoints, PolarisEndpoints.V1_CREATE_GENERIC_TABLE); CreateGenericTableRESTRequest request = new CreateGenericTableRESTRequest(identifier.name(), format, null, props); @@ -164,8 +164,8 @@ public GenericTable createTable( return response.getTable(); } - public GenericTable loadTable(TableIdentifier identifier) { - Endpoint.check(endpoints, PolarisEndpoints.V1_LOAD_GENERIC_TABLE); + public GenericTable loadGenericTable(TableIdentifier identifier) { + // Endpoint.check(endpoints, PolarisEndpoints.V1_LOAD_GENERIC_TABLE); PolarisCatalogUtils.checkIdentifierIsValid(identifier); LoadGenericTableRESTResponse response = restClient diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java index 188a149fb5..a312c4a522 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -58,7 +58,7 @@ public String name() { @Override public Table loadTable(Identifier identifier) throws NoSuchTableException { GenericTable genericTable = - this.restCatalog.loadTable(Spark3Util.identifierToTableIdentifier(identifier)); + this.restCatalog.loadGenericTable(Spark3Util.identifierToTableIdentifier(identifier)); return PolarisCatalogUtils.loadSparkTable(genericTable); } @@ -71,7 +71,7 @@ public Table createTable( throws TableAlreadyExistsException, NoSuchNamespaceException { String format = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); GenericTable genericTable = - this.restCatalog.createTable( + this.restCatalog.createGenericTable( Spark3Util.identifierToTableIdentifier(identifier), format, properties); return PolarisCatalogUtils.loadSparkTable(genericTable); } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index 6d26587157..eac78f5c0f 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -21,6 +21,9 @@ import com.google.common.collect.Maps; import java.lang.reflect.Field; import java.util.Map; + +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchNamespaceException; @@ -93,9 +96,17 @@ public static Table loadSparkTable(GenericTable genericTable) { */ public static OAuth2Util.AuthSession getAuthSession(SparkCatalog sparkCatalog) { try { - Field icebergRestCatalogField = sparkCatalog.getClass().getDeclaredField("icebergCatalog"); - icebergRestCatalogField.setAccessible(true); - RESTCatalog icebergRestCatalog = (RESTCatalog) icebergRestCatalogField.get(sparkCatalog); + Field icebergCatalogField = sparkCatalog.getClass().getDeclaredField("icebergCatalog"); + icebergCatalogField.setAccessible(true); + Catalog icebergCatalog = (Catalog) icebergCatalogField.get(sparkCatalog); + RESTCatalog icebergRestCatalog; + if (icebergCatalog instanceof CachingCatalog) { + Field catalogField = icebergCatalog.getClass().getDeclaredField("catalog"); + catalogField.setAccessible(true); + icebergRestCatalog = (RESTCatalog) catalogField.get(icebergCatalog); + } else { + icebergRestCatalog = (RESTCatalog) icebergCatalog; + } Field sessionCatalogField = icebergRestCatalog.getClass().getDeclaredField("sessionCatalog"); sessionCatalogField.setAccessible(true); From 98118c501f6f7bd3888e1efc21dea2ff68b9695e Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 23:21:29 -0700 Subject: [PATCH 08/29] fix build --- plugins/spark/v3.5/build.gradle.kts | 11 ++++---- .../polaris/spark/PolarisSparkCatalog.java | 25 +++++++++++++------ .../spark/utils/PolarisCatalogUtils.java | 1 - 3 files changed, 22 insertions(+), 15 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index a0b4ca0a71..0479197514 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -50,6 +50,11 @@ dependencies { implementation(project(":polaris-api-catalog-service")) implementation(project(":polaris-core")) { exclude("org.apache.iceberg", "*") } + implementation("org.apache.iceberg:iceberg-core:${icebergVersion}") + implementation("com.fasterxml.jackson.core:jackson-annotations") + implementation("com.fasterxml.jackson.core:jackson-core") + implementation("com.fasterxml.jackson.core:jackson-databind") + implementation( "org.apache.iceberg:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}:${icebergVersion}" ) { @@ -59,12 +64,6 @@ dependencies { exclude("org.apache.iceberg.hadoop", "*") } - implementation("org.apache.iceberg:iceberg-core:${icebergVersion}") - - implementation("com.fasterxml.jackson.core:jackson-annotations") - implementation("com.fasterxml.jackson.core:jackson-core") - implementation("com.fasterxml.jackson.core:jackson-databind") - compileOnly("org.apache.spark:spark-sql_${scalaVersion}:${spark35Version}") { // exclude log4j dependencies exclude("org.apache.logging.log4j", "log4j-slf4j2-impl") diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java index a312c4a522..3a24fefd80 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -19,6 +19,7 @@ package org.apache.polaris.spark; import java.util.Map; +import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.spark.Spark3Util; import org.apache.polaris.service.types.GenericTable; import org.apache.polaris.spark.utils.PolarisCatalogUtils; @@ -57,9 +58,13 @@ public String name() { @Override public Table loadTable(Identifier identifier) throws NoSuchTableException { - GenericTable genericTable = - this.restCatalog.loadGenericTable(Spark3Util.identifierToTableIdentifier(identifier)); - return PolarisCatalogUtils.loadSparkTable(genericTable); + try { + GenericTable genericTable = + this.restCatalog.loadGenericTable(Spark3Util.identifierToTableIdentifier(identifier)); + return PolarisCatalogUtils.loadSparkTable(genericTable); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(identifier); + } } @Override @@ -69,11 +74,15 @@ public Table createTable( Transform[] transforms, Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { - String format = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); - GenericTable genericTable = - this.restCatalog.createGenericTable( - Spark3Util.identifierToTableIdentifier(identifier), format, properties); - return PolarisCatalogUtils.loadSparkTable(genericTable); + try { + String format = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); + GenericTable genericTable = + this.restCatalog.createGenericTable( + Spark3Util.identifierToTableIdentifier(identifier), format, properties); + return PolarisCatalogUtils.loadSparkTable(genericTable); + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistsException(identifier); + } } @Override diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index eac78f5c0f..6e3b6b51b9 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -21,7 +21,6 @@ import com.google.common.collect.Maps; import java.lang.reflect.Field; import java.util.Map; - import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; From 468007176a152d6c590bb4eb00443a7f91ed5a18 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 23:22:43 -0700 Subject: [PATCH 09/29] remove chagne --- .../core/rest/PolarisResourcePaths.java | 62 ------------------- 1 file changed, 62 deletions(-) delete mode 100644 polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java diff --git a/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java b/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java deleted file mode 100644 index 159a1a0148..0000000000 --- a/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.rest; - -import com.google.common.base.Joiner; -import java.util.Map; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.rest.RESTUtil; - -public class PolarisResourcePaths { - private static final Joiner SLASH = Joiner.on("/").skipNulls(); - public static final String PREFIX = "prefix"; - - // Generic Table endpoints - public static final String V1_GENERIC_TABLES = - "polaris/v1/{prefix}/namespaces/{namespace}/generic-tables"; - public static final String V1_GENERIC_TABLE = - "polaris/v1/{prefix}/namespaces/{namespace}/generic-tables/{generic-table}"; - - private final String prefix; - - public PolarisResourcePaths(String prefix) { - this.prefix = prefix; - } - - public static PolarisResourcePaths forCatalogProperties(Map properties) { - return new PolarisResourcePaths(properties.get(PREFIX)); - } - - public String genericTables(Namespace ns) { - return SLASH.join( - "polaris", "v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "generic-tables"); - } - - public String genericTable(TableIdentifier ident) { - return SLASH.join( - "polaris", - "v1", - prefix, - "namespaces", - RESTUtil.encodeNamespace(ident.namespace()), - "generic-tables", - RESTUtil.encodeString(ident.name())); - } -} From 12028f1ae710423ee15d1d32b593837cdda54489 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 2 Apr 2025 23:26:38 -0700 Subject: [PATCH 10/29] rebase change --- .../core/rest/PolarisResourcePaths.java | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java diff --git a/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java b/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java new file mode 100644 index 0000000000..159a1a0148 --- /dev/null +++ b/polaris-core/src/main/java/org/apache/polaris/core/rest/PolarisResourcePaths.java @@ -0,0 +1,62 @@ +/* + * 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.rest; + +import com.google.common.base.Joiner; +import java.util.Map; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.RESTUtil; + +public class PolarisResourcePaths { + private static final Joiner SLASH = Joiner.on("/").skipNulls(); + public static final String PREFIX = "prefix"; + + // Generic Table endpoints + public static final String V1_GENERIC_TABLES = + "polaris/v1/{prefix}/namespaces/{namespace}/generic-tables"; + public static final String V1_GENERIC_TABLE = + "polaris/v1/{prefix}/namespaces/{namespace}/generic-tables/{generic-table}"; + + private final String prefix; + + public PolarisResourcePaths(String prefix) { + this.prefix = prefix; + } + + public static PolarisResourcePaths forCatalogProperties(Map properties) { + return new PolarisResourcePaths(properties.get(PREFIX)); + } + + public String genericTables(Namespace ns) { + return SLASH.join( + "polaris", "v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "generic-tables"); + } + + public String genericTable(TableIdentifier ident) { + return SLASH.join( + "polaris", + "v1", + prefix, + "namespaces", + RESTUtil.encodeNamespace(ident.namespace()), + "generic-tables", + RESTUtil.encodeString(ident.name())); + } +} From c93e99bdd8a58d2d25953fe5776826d33d64b150 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 3 Apr 2025 17:17:15 -0700 Subject: [PATCH 11/29] add in memory classes --- plugins/pluginlibs.versions.toml | 3 + plugins/spark/v3.5/build.gradle.kts | 13 ++- .../apache/polaris/spark/PolarisCatalog.java | 36 ++++++++ .../polaris/spark/PolarisInMemoryCatalog.java | 88 +++++++++++++++++++ .../polaris/spark/PolarisRESTCatalog.java | 24 ++--- .../polaris/spark/PolarisSparkCatalog.java | 18 ++-- .../apache/polaris/spark/SparkCatalog.java | 64 +++++++++++--- .../rest/CreateGenericTableRESTRequest.java | 5 ++ .../rest/LoadGenericTableRESTResponse.java | 5 ++ .../polaris/spark/utils/DeltaHelper.java | 19 ++-- .../spark/utils/PolarisCatalogUtils.java | 29 +++--- .../polaris/spark/InMemoryDeltaCatalog.java | 29 ++++++ .../polaris/spark/SparkCatalogTest.java | 23 +++-- 13 files changed, 296 insertions(+), 60 deletions(-) create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java create mode 100644 plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java create mode 100644 plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java diff --git a/plugins/pluginlibs.versions.toml b/plugins/pluginlibs.versions.toml index 0a4a515e5c..e48f6ef45a 100644 --- a/plugins/pluginlibs.versions.toml +++ b/plugins/pluginlibs.versions.toml @@ -20,3 +20,6 @@ [versions] iceberg = "1.8.1" spark35 = "3.5.5" +scala212 = "2.12.19" +scala213 = "2.13.15" + diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 0479197514..2ab1d55164 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -41,6 +41,13 @@ val scalaVersion = getAndUseScalaVersionForProject() val icebergVersion = pluginlibs.versions.iceberg.get() val spark35Version = pluginlibs.versions.spark35.get() +val scalaLibraryVersion = + if (scalaVersion.equals("2.12")) { + pluginlibs.versions.scala212.get() + } else { + pluginlibs.versions.scala213.get() + } + dependencies { implementation(project(":polaris-api-iceberg-service")) { // exclude the iceberg dependencies, use the ones pulled @@ -64,6 +71,7 @@ dependencies { exclude("org.apache.iceberg.hadoop", "*") } + compileOnly("org.scala-lang:scala-library:${scalaLibraryVersion}") compileOnly("org.apache.spark:spark-sql_${scalaVersion}:${spark35Version}") { // exclude log4j dependencies exclude("org.apache.logging.log4j", "log4j-slf4j2-impl") @@ -106,7 +114,10 @@ tasks.register("createPolarisSparkJar") { // Optimization: Minimize the JAR (remove unused classes from dependencies) // The iceberg-spark-runtime plugin is always packaged along with our polaris-spark plugin, // therefore excluded from the optimization. - // minimize { exclude(dependency("org.apache.iceberg:iceberg-spark-runtime-*.*")) } + minimize { + exclude(dependency("org.apache.iceberg:iceberg-spark-runtime-*.*")) + exclude(dependency("org.apache.iceberg:iceberg-core*.*")) + } } tasks.withType(Jar::class).named("sourcesJar") { dependsOn("createPolarisSparkJar") } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java new file mode 100644 index 0000000000..f35d2dc83c --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java @@ -0,0 +1,36 @@ +/* + * 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.spark; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.polaris.service.types.GenericTable; + +public interface PolarisCatalog { + List listGenericTables(Namespace ns); + + GenericTable loadGenericTable(TableIdentifier identifier); + + boolean dropGenericTable(TableIdentifier identifier); + + GenericTable createGenericTable( + TableIdentifier identifier, String format, Map props); +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java new file mode 100644 index 0000000000..ed87535d27 --- /dev/null +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java @@ -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.spark; + +import java.util.*; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.polaris.service.types.GenericTable; + +/** InMemory implementation for the Polaris Catalog. This class is mainly used by testing. */ +public class PolarisInMemoryCatalog implements PolarisCatalog { + private final ConcurrentMap genericTables; + + public PolarisInMemoryCatalog() { + this.genericTables = Maps.newConcurrentMap(); + } + + @Override + public List listGenericTables(Namespace ns) { + return this.genericTables.keySet().stream() + .filter(t -> t.namespace().equals(ns)) + .sorted(Comparator.comparing(TableIdentifier::toString)) + .collect(Collectors.toList()); + } + + @Override + public GenericTable loadGenericTable(TableIdentifier identifier) { + GenericTable table = this.genericTables.get(identifier); + if (table == null) { + throw new NoSuchTableException("Generic table does not exist: %s", identifier); + } + + return table; + } + + @Override + public boolean dropGenericTable(TableIdentifier identifier) { + synchronized (this) { + if (null == this.genericTables.remove(identifier)) { + return false; + } + } + + return true; + } + + @Override + public GenericTable createGenericTable( + TableIdentifier identifier, String format, Map props) { + synchronized (this) { + if (this.genericTables.containsKey(identifier)) { + throw new AlreadyExistsException("Generic table %s already exists", identifier); + } + this.genericTables.compute( + identifier, + (k, table) -> { + return GenericTable.builder() + .setName(k.name()) + .setFormat(format) + .setProperties(props) + .build(); + }); + } + + return this.genericTables.get(identifier); + } +} diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index b4f996a3c6..cfc8516036 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -41,12 +41,8 @@ import org.apache.polaris.spark.rest.CreateGenericTableRESTRequest; import org.apache.polaris.spark.rest.LoadGenericTableRESTResponse; import org.apache.polaris.spark.utils.PolarisCatalogUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class PolarisRESTCatalog implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(PolarisRESTCatalog.class); +public class PolarisRESTCatalog implements PolarisCatalog, Closeable { public static final String REST_PAGE_SIZE = "rest-page-size"; private RESTClient restClient = null; @@ -56,14 +52,8 @@ public class PolarisRESTCatalog implements Closeable { private PolarisResourcePaths paths = null; private Integer pageSize = null; - // TODO: update to use the predefined GENERIC_TABLE_ENDPOINTS - private static final Set DEFAULT_ENDPOINTS = - ImmutableSet.builder() - .add(PolarisEndpoints.V1_CREATE_GENERIC_TABLE) - .add(PolarisEndpoints.V1_LOAD_GENERIC_TABLE) - .add(PolarisEndpoints.V1_DELETE_GENERIC_TABLE) - .add(PolarisEndpoints.V1_LIST_GENERIC_TABLES) - .build(); + // the default endpoints to config if server doesn't specify the 'endpoints' configuration. + private static final Set DEFAULT_ENDPOINTS = PolarisEndpoints.GENERIC_TABLE_ENDPOINTS; public void initialize(Map unresolved, OAuth2Util.AuthSession catalogAuth) { Preconditions.checkArgument(unresolved != null, "Invalid configuration: null"); @@ -137,17 +127,20 @@ public void close() throws IOException { } } + @Override public List listGenericTables(Namespace ns) { throw new UnsupportedOperationException("listTables not supported"); } + @Override public boolean dropGenericTable(TableIdentifier identifier) { throw new UnsupportedOperationException("dropTable not supported"); } + @Override public GenericTable createGenericTable( TableIdentifier identifier, String format, Map props) { - // Endpoint.check(endpoints, PolarisEndpoints.V1_CREATE_GENERIC_TABLE); + Endpoint.check(endpoints, PolarisEndpoints.V1_CREATE_GENERIC_TABLE); CreateGenericTableRESTRequest request = new CreateGenericTableRESTRequest(identifier.name(), format, null, props); @@ -164,8 +157,9 @@ public GenericTable createGenericTable( return response.getTable(); } + @Override public GenericTable loadGenericTable(TableIdentifier identifier) { - // Endpoint.check(endpoints, PolarisEndpoints.V1_LOAD_GENERIC_TABLE); + Endpoint.check(endpoints, PolarisEndpoints.V1_LOAD_GENERIC_TABLE); PolarisCatalogUtils.checkIdentifierIsValid(identifier); LoadGenericTableRESTResponse response = restClient diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java index 3a24fefd80..8147e197af 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -33,17 +33,19 @@ import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +/** + * A spark TableCatalog Implementation interacts with Polaris specific APIs only. The APIs it + * interacts with is generic table APIs, and all table operations performed in this class are + * expected to be for non-iceberg tables. + */ public class PolarisSparkCatalog implements TableCatalog { - private static final Logger LOG = LoggerFactory.getLogger(PolarisSparkCatalog.class); - private PolarisRESTCatalog restCatalog = null; + private PolarisCatalog polarisCatalog = null; private String catalogName = null; - public PolarisSparkCatalog(PolarisRESTCatalog restCatalog) { - this.restCatalog = restCatalog; + public PolarisSparkCatalog(PolarisCatalog polarisCatalog) { + this.polarisCatalog = polarisCatalog; } @Override @@ -60,7 +62,7 @@ public String name() { public Table loadTable(Identifier identifier) throws NoSuchTableException { try { GenericTable genericTable = - this.restCatalog.loadGenericTable(Spark3Util.identifierToTableIdentifier(identifier)); + this.polarisCatalog.loadGenericTable(Spark3Util.identifierToTableIdentifier(identifier)); return PolarisCatalogUtils.loadSparkTable(genericTable); } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { throw new NoSuchTableException(identifier); @@ -77,7 +79,7 @@ public Table createTable( try { String format = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); GenericTable genericTable = - this.restCatalog.createGenericTable( + this.polarisCatalog.createGenericTable( Spark3Util.identifierToTableIdentifier(identifier), format, properties); return PolarisCatalogUtils.loadSparkTable(genericTable); } catch (AlreadyExistsException e) { diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index 4446fd419b..815ba71d30 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -47,9 +47,13 @@ import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +/** + * SparkCatalog Implementation that is able to interact with both Iceberg SparkCatalog and Polaris + * SparkCatalog. All namespaces and view related operations continue goes through the Iceberg + * SparkCatalog. For table operations, depends on the table format, the operation can be achieved + * with interaction with both Iceberg and Polaris SparkCatalog. + */ public class SparkCatalog implements StagingTableCatalog, TableCatalog, @@ -69,14 +73,32 @@ public String name() { return catalogName; } - @Override - public void initialize(String name, CaseInsensitiveStringMap options) { + /** + * Initialize an inMemory Iceberg Catalog and Polaris Catalog. NOTE: This should only be used by + * testing. + */ + private void initInMemoryCatalog(String name, CaseInsensitiveStringMap options) { String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); - // TODO: relax this once AuthManager reuse enabled - if (catalogImpl != null) { - throw new UnsupportedOperationException( - "Customized catalog implementation is currently not supported!"); + if (catalogImpl == null) { + throw new IllegalStateException( + "Missing catalog-impl configuration, required when InMemory Catalog mode is enabled"); } + // initialize the icebergSparkCatalog with configured CATALOG_IMPL + this.icebergsSparkCatalog = new org.apache.iceberg.spark.SparkCatalog(); + this.icebergsSparkCatalog.initialize(name, options); + + // initialize the polarisSparkCatalog with PolarisSparkCatalog + PolarisInMemoryCatalog inMemoryCatalog = new PolarisInMemoryCatalog(); + this.polarisSparkCatalog = new PolarisSparkCatalog(inMemoryCatalog); + this.polarisSparkCatalog.initialize(name, options); + } + + /** + * Initialize REST Catalog for Iceberg and Polaris, this is the only catalog type supported by + * Polaris at this moment. + */ + private void initRESTCatalog(String name, CaseInsensitiveStringMap options) { + // TODO: relax this in the future String catalogType = PropertyUtil.propertyAsString( options, CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); @@ -85,7 +107,12 @@ public void initialize(String name, CaseInsensitiveStringMap options) { "Only rest catalog type is supported, but got catalog type: " + catalogType); } - this.catalogName = name; + String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); + if (catalogImpl != null) { + throw new UnsupportedOperationException( + "Customized catalog implementation is currently not supported!"); + } + // initialize the icebergSparkCatalog this.icebergsSparkCatalog = new org.apache.iceberg.spark.SparkCatalog(); this.icebergsSparkCatalog.initialize(name, options); @@ -97,8 +124,23 @@ public void initialize(String name, CaseInsensitiveStringMap options) { restCatalog.initialize(options, catalogAuth); this.polarisSparkCatalog = new PolarisSparkCatalog(restCatalog); this.polarisSparkCatalog.initialize(name, options); + } + + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + this.catalogName = name; + boolean enableInMemoryCatalog = + options.containsKey(PolarisCatalogUtils.ENABLE_IN_MEMORY_CATALOG_KEY) + && options + .get(PolarisCatalogUtils.ENABLE_IN_MEMORY_CATALOG_KEY) + .equalsIgnoreCase("true"); + if (enableInMemoryCatalog) { + initInMemoryCatalog(name, options); + } else { + initRESTCatalog(name, options); + } - this.deltaHelper = new DeltaHelper(); + this.deltaHelper = new DeltaHelper(options); } @Override @@ -118,6 +160,8 @@ public Table createTable( if (PolarisCatalogUtils.useIceberg(provider)) { return this.icebergsSparkCatalog.createTable(ident, schema, transforms, properties); } else if (PolarisCatalogUtils.useDelta(provider)) { + // For delta table, we load the delta catalog to help dealing with the + // delta log creation. TableCatalog deltaCatalog = deltaHelper.loadDeltaCatalog(this.polarisSparkCatalog); return deltaCatalog.createTable(ident, schema, transforms, properties); } else { diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java index ce0a5c17af..4ec348a80a 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/CreateGenericTableRESTRequest.java @@ -24,6 +24,11 @@ import org.apache.iceberg.rest.RESTRequest; import org.apache.polaris.service.types.CreateGenericTableRequest; +/** + * RESTRequest definition for CreateGenericTable which extends the iceberg RESTRequest. This is + * currently required because the Iceberg HTTPClient requires the request and response to be a class + * of RESTRequest and RESTResponse. + */ public class CreateGenericTableRESTRequest extends CreateGenericTableRequest implements RESTRequest { diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java index 799942cb99..68c738dae4 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/rest/LoadGenericTableRESTResponse.java @@ -24,6 +24,11 @@ import org.apache.polaris.service.types.GenericTable; import org.apache.polaris.service.types.LoadGenericTableResponse; +/** + * RESTResponse definition for LoadGenericTable which extends the iceberg RESTResponse. This is + * currently required because the Iceberg HTTPClient requires the request and response to be a class + * of RESTRequest and RESTResponse. + */ public class LoadGenericTableRESTResponse extends LoadGenericTableResponse implements RESTResponse { @JsonCreator diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java index 48c3bd50d9..8f1d556e1d 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -24,15 +24,25 @@ import org.apache.polaris.spark.PolarisSparkCatalog; import org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension; import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DeltaHelper { private static final Logger LOG = LoggerFactory.getLogger(DeltaHelper.class); - private static final String DELTA_CATALOG_CLASS = + public static final String DELTA_CATALOG_IMPL_KEY = "delta-catalog-impl"; + private static final String DEFAULT_DELTA_CATALOG_CLASS = "org.apache.spark.sql.delta.catalog.DeltaCatalog"; + private TableCatalog deltaCatalog = null; + private String deltaCatalogImpl = DEFAULT_DELTA_CATALOG_CLASS; + + public DeltaHelper(CaseInsensitiveStringMap options) { + if (options.get(DELTA_CATALOG_IMPL_KEY) != null) { + this.deltaCatalogImpl = options.get(DELTA_CATALOG_IMPL_KEY); + } + } public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { if (this.deltaCatalog != null) { @@ -41,11 +51,10 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { DynConstructors.Ctor ctor; try { - ctor = DynConstructors.builder(TableCatalog.class).impl(DELTA_CATALOG_CLASS).buildChecked(); + ctor = DynConstructors.builder(TableCatalog.class).impl(deltaCatalogImpl).buildChecked(); } catch (NoSuchMethodException e) { throw new IllegalArgumentException( - String.format( - "Cannot initialize Delta Catalog %s: %s", DELTA_CATALOG_CLASS, e.getMessage()), + String.format("Cannot initialize Delta Catalog %s: %s", deltaCatalogImpl, e.getMessage()), e); } @@ -55,7 +64,7 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { throw new IllegalArgumentException( String.format( "Cannot initialize Delta Catalog, %s does not implement Table Catalog.", - DELTA_CATALOG_CLASS), + deltaCatalogImpl), e); } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index 6e3b6b51b9..9b982ea242 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -23,9 +23,7 @@ import java.util.Map; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.rest.RESTCatalog; import org.apache.iceberg.rest.RESTSessionCatalog; @@ -45,11 +43,8 @@ public class PolarisCatalogUtils { public static final String TABLE_PROVIDER_KEY = "provider"; public static final String TABLE_PATH_KEY = "path"; - public static void checkNamespaceIsValid(Namespace namespace) { - if (namespace.isEmpty()) { - throw new NoSuchNamespaceException("Invalid namespace: %s", namespace); - } - } + // whether enable the inMemory catalogs, used by testing. + public static final String ENABLE_IN_MEMORY_CATALOG_KEY = "enable_in_memory_catalog"; public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { if (tableIdentifier.namespace().isEmpty()) { @@ -57,29 +52,37 @@ public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { } } + /** Check whether the table provider is iceberg. */ public static boolean useIceberg(String provider) { return provider == null || "iceberg".equalsIgnoreCase(provider); } + /** Check whether the table provider is delta. */ public static boolean useDelta(String provider) { return "delta".equalsIgnoreCase(provider); } + /** + * Load spark table using DataSourceV2. + * + * @return V2Table if DataSourceV2 is available for the table format. For delta table, it returns + * DeltaTableV2. + */ public static Table loadSparkTable(GenericTable genericTable) { SparkSession sparkSession = SparkSession.active(); TableProvider provider = DataSource.lookupDataSourceV2(genericTable.getFormat(), sparkSession.sessionState().conf()) .get(); Map properties = genericTable.getProperties(); - boolean hasLocationClause = - properties.containsKey(TableCatalog.PROP_LOCATION) - && (properties.get(TableCatalog.PROP_LOCATION) != null); - boolean hasPathClause = - properties.containsKey(TABLE_PATH_KEY) && (properties.get(TABLE_PATH_KEY) != null); + boolean hasLocationClause = properties.get(TableCatalog.PROP_LOCATION) != null; + boolean hasPathClause = properties.get(TABLE_PATH_KEY) != null; Map tableProperties = Maps.newHashMap(); tableProperties.putAll(properties); if (!hasPathClause && hasLocationClause) { - // DataSourceV2 requires the path for table loading + // DataSourceV2 requires the path property on table loading. However, spark today + // doesn't create the corresponding path property if the path keyword is not + // provided by user when location is provided. Here, we duplicate the location + // property as path to make sure the table can be loaded. tableProperties.put(TABLE_PATH_KEY, properties.get(TableCatalog.PROP_LOCATION)); } CaseInsensitiveStringMap property_map = new CaseInsensitiveStringMap(tableProperties); diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java new file mode 100644 index 0000000000..7080812808 --- /dev/null +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java @@ -0,0 +1,29 @@ +/* + * 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.spark; + +import org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension; + +public class InMemoryDeltaCatalog extends DelegatingCatalogExtension { + private boolean isUnityCatalog = false; + + private void isUnityCatalog$lzycompute() { + isUnityCatalog = true; + } +} diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index 70e9b00c52..2b6edae679 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -38,6 +38,15 @@ import org.apache.spark.sql.connector.catalog.*; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.polaris.spark.utils.DeltaHelper; +import org.apache.polaris.spark.utils.PolarisCatalogUtils; +import org.apache.spark.SparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -61,7 +70,9 @@ public void setup() throws Exception { Map catalogConfig = Maps.newHashMap(); catalogConfig.put(CATALOG_IMPL, "org.apache.iceberg.inmemory.InMemoryCatalog"); catalogConfig.put("cache-enabled", "false"); - + catalogConfig.put(PolarisCatalogUtils.ENABLE_IN_MEMORY_CATALOG_KEY, "true"); + catalogConfig.put( + DeltaHelper.DELTA_CATALOG_IMPL_KEY, "org.apache.polaris.spark.InMemoryDeltaCatalog"); catalog = new SparkCatalog(); Configuration conf = new Configuration(); try (MockedStatic mockedStaticSparkSession = @@ -83,8 +94,10 @@ public void setup() throws Exception { catalog.createNamespace(defaultNS, Maps.newHashMap()); } + @Test - void testCreateAndLoadNamespace() throws Exception { + void testCreateAndLoadNamespace() + throws NoSuchNamespaceException, NamespaceAlreadyExistsException { String[] namespace = new String[] {"ns1"}; Map metadata = Maps.newHashMap(); metadata.put("key1", "value1"); @@ -292,11 +305,6 @@ public void testUnsupportedOperations() { Identifier identifier = Identifier.of(namespace, "table1"); Identifier new_identifier = Identifier.of(namespace, "table2"); // table methods - assertThatThrownBy(() -> catalog.loadTable(identifier)) - .isInstanceOf(UnsupportedOperationException.class); - assertThatThrownBy( - () -> catalog.createTable(identifier, Mockito.mock(StructType.class), null, null)) - .isInstanceOf(UnsupportedOperationException.class); assertThatThrownBy(() -> catalog.alterTable(identifier)) .isInstanceOf(UnsupportedOperationException.class); assertThatThrownBy(() -> catalog.dropTable(identifier)) @@ -308,6 +316,5 @@ public void testUnsupportedOperations() { assertThatThrownBy(() -> catalog.invalidateTable(identifier)) .isInstanceOf(UnsupportedOperationException.class); assertThatThrownBy(() -> catalog.purgeTable(identifier)) - .isInstanceOf(UnsupportedOperationException.class); } } From 314b9cc6904416f12378970ac5a1599bade7b5ee Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Fri, 4 Apr 2025 18:43:45 -0700 Subject: [PATCH 12/29] fix inMemory testing --- plugins/spark/v3.5/build.gradle.kts | 2 +- .../polaris/spark/PolarisRESTCatalog.java | 30 ++-- .../apache/polaris/spark/SparkCatalog.java | 35 +--- .../spark/utils/PolarisCatalogUtils.java | 3 - .../polaris/spark/PolarisInMemoryCatalog.java | 16 +- .../polaris/spark/SparkCatalogTest.java | 158 ++++++++++++++++-- 6 files changed, 184 insertions(+), 60 deletions(-) rename plugins/spark/v3.5/src/{main => test}/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java (78%) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 2ab1d55164..66bf0dbe85 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -42,7 +42,7 @@ val icebergVersion = pluginlibs.versions.iceberg.get() val spark35Version = pluginlibs.versions.spark35.get() val scalaLibraryVersion = - if (scalaVersion.equals("2.12")) { + if (scalaVersion == "2.12") { pluginlibs.versions.scala212.get() } else { pluginlibs.versions.scala213.get() diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index cfc8516036..aae7260be5 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -22,9 +22,11 @@ import com.google.common.collect.ImmutableSet; import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -45,6 +47,8 @@ public class PolarisRESTCatalog implements PolarisCatalog, Closeable { public static final String REST_PAGE_SIZE = "rest-page-size"; + private final Function, RESTClient> clientBuilder; + private RESTClient restClient = null; private CloseableGroup closeables = null; private Set endpoints; @@ -55,6 +59,14 @@ public class PolarisRESTCatalog implements PolarisCatalog, Closeable { // the default endpoints to config if server doesn't specify the 'endpoints' configuration. private static final Set DEFAULT_ENDPOINTS = PolarisEndpoints.GENERIC_TABLE_ENDPOINTS; + public PolarisRESTCatalog() { + this(config -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()); + } + + public PolarisRESTCatalog(Function, RESTClient> clientBuilder) { + this.clientBuilder = clientBuilder; + } + public void initialize(Map unresolved, OAuth2Util.AuthSession catalogAuth) { Preconditions.checkArgument(unresolved != null, "Invalid configuration: null"); @@ -64,14 +76,14 @@ public void initialize(Map unresolved, OAuth2Util.AuthSession ca // TODO: switch to use authManager once iceberg dependency is updated to 1.9.0 this.catalogAuth = catalogAuth; - this.restClient = - HTTPClient.builder(props) - .uri(props.get(CatalogProperties.URI)) - .build() - .withAuthSession(catalogAuth); + ConfigResponse config; + try (RESTClient initClient = clientBuilder.apply(props).withAuthSession(catalogAuth)) { + config = fetchConfig(initClient, catalogAuth.headers(), props); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close HTTP client", e); + } // call getConfig to get the server configurations - ConfigResponse config = fetchConfig(this.restClient, catalogAuth.headers(), props); Map mergedProps = config.merge(props); if (config.endpoints().isEmpty()) { this.endpoints = DEFAULT_ENDPOINTS; @@ -80,11 +92,7 @@ public void initialize(Map unresolved, OAuth2Util.AuthSession ca } this.paths = PolarisResourcePaths.forCatalogProperties(mergedProps); - this.restClient = - HTTPClient.builder(mergedProps) - .uri(mergedProps.get(CatalogProperties.URI)) - .build() - .withAuthSession(catalogAuth); + this.restClient = clientBuilder.apply(mergedProps).withAuthSession(catalogAuth); this.pageSize = PropertyUtil.propertyAsNullableInt(mergedProps, REST_PAGE_SIZE); if (pageSize != null) { diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index 815ba71d30..ae456a09ef 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -66,33 +66,13 @@ public class SparkCatalog private org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; private PolarisSparkCatalog polarisSparkCatalog = null; - private DeltaHelper deltaHelper = null; + protected DeltaHelper deltaHelper = null; @Override public String name() { return catalogName; } - /** - * Initialize an inMemory Iceberg Catalog and Polaris Catalog. NOTE: This should only be used by - * testing. - */ - private void initInMemoryCatalog(String name, CaseInsensitiveStringMap options) { - String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); - if (catalogImpl == null) { - throw new IllegalStateException( - "Missing catalog-impl configuration, required when InMemory Catalog mode is enabled"); - } - // initialize the icebergSparkCatalog with configured CATALOG_IMPL - this.icebergsSparkCatalog = new org.apache.iceberg.spark.SparkCatalog(); - this.icebergsSparkCatalog.initialize(name, options); - - // initialize the polarisSparkCatalog with PolarisSparkCatalog - PolarisInMemoryCatalog inMemoryCatalog = new PolarisInMemoryCatalog(); - this.polarisSparkCatalog = new PolarisSparkCatalog(inMemoryCatalog); - this.polarisSparkCatalog.initialize(name, options); - } - /** * Initialize REST Catalog for Iceberg and Polaris, this is the only catalog type supported by * Polaris at this moment. @@ -128,18 +108,7 @@ private void initRESTCatalog(String name, CaseInsensitiveStringMap options) { @Override public void initialize(String name, CaseInsensitiveStringMap options) { - this.catalogName = name; - boolean enableInMemoryCatalog = - options.containsKey(PolarisCatalogUtils.ENABLE_IN_MEMORY_CATALOG_KEY) - && options - .get(PolarisCatalogUtils.ENABLE_IN_MEMORY_CATALOG_KEY) - .equalsIgnoreCase("true"); - if (enableInMemoryCatalog) { - initInMemoryCatalog(name, options); - } else { - initRESTCatalog(name, options); - } - + initRESTCatalog(name, options); this.deltaHelper = new DeltaHelper(options); } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index 9b982ea242..44dda0ca83 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -43,9 +43,6 @@ public class PolarisCatalogUtils { public static final String TABLE_PROVIDER_KEY = "provider"; public static final String TABLE_PATH_KEY = "path"; - // whether enable the inMemory catalogs, used by testing. - public static final String ENABLE_IN_MEMORY_CATALOG_KEY = "enable_in_memory_catalog"; - public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { if (tableIdentifier.namespace().isEmpty()) { throw new NoSuchTableException("Invalid table identifier: %s", tableIdentifier); diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java similarity index 78% rename from plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java rename to plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java index ed87535d27..40e98a8bd1 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java @@ -24,12 +24,14 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.inmemory.InMemoryCatalog; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.polaris.service.types.GenericTable; /** InMemory implementation for the Polaris Catalog. This class is mainly used by testing. */ -public class PolarisInMemoryCatalog implements PolarisCatalog { +public class PolarisInMemoryCatalog extends InMemoryCatalog implements PolarisCatalog { private final ConcurrentMap genericTables; public PolarisInMemoryCatalog() { @@ -69,6 +71,18 @@ public boolean dropGenericTable(TableIdentifier identifier) { public GenericTable createGenericTable( TableIdentifier identifier, String format, Map props) { synchronized (this) { + if (!namespaceExists(identifier.namespace())) { + throw new NoSuchNamespaceException( + "Cannot create generic table %s. Namespace does not exist: %s", + identifier, identifier.namespace()); + } + if (listViews(identifier.namespace()).contains(identifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", identifier); + } + if (listTables(identifier.namespace()).contains(identifier)) { + throw new AlreadyExistsException( + "Iceberg table with same name already exists: %s", identifier); + } if (this.genericTables.containsKey(identifier)) { throw new AlreadyExistsException("Generic table %s already exists", identifier); } diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index 2b6edae679..c0ec6c9e50 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.google.common.collect.Maps; import java.util.Arrays; import java.util.Map; @@ -38,23 +39,74 @@ import org.apache.spark.sql.connector.catalog.*; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; +import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.spark.source.SparkTable; import org.apache.polaris.spark.utils.DeltaHelper; import org.apache.polaris.spark.utils.PolarisCatalogUtils; import org.apache.spark.SparkContext; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.catalog.*; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.execution.datasources.DataSource; +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.internal.SessionState; +import org.apache.spark.sql.types.*; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.MockedStatic; import org.mockito.Mockito; +import scala.Option; public class SparkCatalogTest { - private SparkCatalog catalog; + private class InMemoryIcebergSparkCatalog extends org.apache.iceberg.spark.SparkCatalog { + private PolarisInMemoryCatalog inMemoryCatalog = null; + + @Override + protected Catalog buildIcebergCatalog(String name, CaseInsensitiveStringMap options) { + PolarisInMemoryCatalog inMemoryCatalog = new PolarisInMemoryCatalog(); + inMemoryCatalog.initialize(name, options); + + this.inMemoryCatalog = inMemoryCatalog; + + return inMemoryCatalog; + } + + public PolarisInMemoryCatalog getInMemoryCatalog() { + return this.inMemoryCatalog; + } + } + + /** + * And SparkCatalog implementation that uses InMemory catalog implementation for both Iceberg and + * Polaris + */ + private class InMemorySparkCatalog extends SparkCatalog { + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + this.catalogName = name; + // initialize the InMemory icebergSparkCatalog + this.icebergsSparkCatalog = new InMemoryIcebergSparkCatalog(); + this.icebergsSparkCatalog.initialize(name, options); + + // initialize the polarisSparkCatalog with PolarisSparkCatalog + this.polarisSparkCatalog = + new PolarisSparkCatalog( + ((InMemoryIcebergSparkCatalog) this.icebergsSparkCatalog).getInMemoryCatalog()); + this.polarisSparkCatalog.initialize(name, options); + + this.deltaHelper = new DeltaHelper(options); + } + } + + private InMemorySparkCatalog catalog; private String catalogName; private static final String[] defaultNS = new String[] {"ns"}; @@ -70,10 +122,9 @@ public void setup() throws Exception { Map catalogConfig = Maps.newHashMap(); catalogConfig.put(CATALOG_IMPL, "org.apache.iceberg.inmemory.InMemoryCatalog"); catalogConfig.put("cache-enabled", "false"); - catalogConfig.put(PolarisCatalogUtils.ENABLE_IN_MEMORY_CATALOG_KEY, "true"); catalogConfig.put( DeltaHelper.DELTA_CATALOG_IMPL_KEY, "org.apache.polaris.spark.InMemoryDeltaCatalog"); - catalog = new SparkCatalog(); + catalog = new InMemorySparkCatalog(); Configuration conf = new Configuration(); try (MockedStatic mockedStaticSparkSession = Mockito.mockStatic(SparkSession.class); @@ -96,8 +147,7 @@ public void setup() throws Exception { @Test - void testCreateAndLoadNamespace() - throws NoSuchNamespaceException, NamespaceAlreadyExistsException { + void testCreateAndLoadNamespace() throws Exception { String[] namespace = new String[] {"ns1"}; Map metadata = Maps.newHashMap(); metadata.put("key1", "value1"); @@ -251,7 +301,7 @@ void testBasicViewOperations() throws Exception { @Test void testListViews() throws Exception { // create a new namespace under the default NS - String[] namespace = new String[] {"ns", "nsl2"}; + String[] namespace = new String[]{"ns", "nsl2"}; catalog.createNamespace(namespace, Maps.newHashMap()); // table schema StructType schema = new StructType().add("id", "long").add("name", "string"); @@ -269,10 +319,10 @@ void testListViews() throws Exception { new String[0], Maps.newHashMap()); // create two views under ns.nsl2 - String[] nsl2ViewNames = new String[] {"test-view2", "test-view3"}; + String[] nsl2ViewNames = new String[]{"test-view2", "test-view3"}; String[] nsl2ViewSQLs = - new String[] { - "select id from test-table where id == 3", "select id from test-table where id < 3" + new String[]{ + "select id from test-table where id == 3", "select id from test-table where id < 3" }; for (int i = 0; i < nsl2ViewNames.length; i++) { catalog.createView( @@ -299,6 +349,92 @@ void testListViews() throws Exception { } } + void testCreateAndLoadIcebergTable() throws Exception { + String[] namespace = new String[] {"ns"}; + Identifier identifier = Identifier.of(namespace, "iceberg-table"); + Map properties = Maps.newHashMap(); + properties.put(PolarisCatalogUtils.TABLE_PROVIDER_KEY, "iceberg"); + properties.put(TableCatalog.PROP_LOCATION, "file:///tmp/path/to/table/"); + StructType schema = new StructType().add("boolType", "boolean"); + + assertThatThrownBy(() -> catalog.createTable(identifier, schema, new Transform[0], properties)) + .isInstanceOf(org.apache.iceberg.exceptions.NoSuchNamespaceException.class); + + catalog.createNamespace(namespace, Maps.newHashMap()); + + Table createdTable = catalog.createTable(identifier, schema, new Transform[0], properties); + assertThat(createdTable).isInstanceOf(SparkTable.class); + + // load the table + Table table = catalog.loadTable(identifier); + // verify iceberg SparkTable is loaded + assertThat(table).isInstanceOf(SparkTable.class); + + // verify create table with the same identifier fails with spark TableAlreadyExistsException + StructType newSchema = new StructType().add("LongType", "Long"); + Map newProperties = Maps.newHashMap(); + newProperties.put(PolarisCatalogUtils.TABLE_PROVIDER_KEY, "iceberg"); + newProperties.put(TableCatalog.PROP_LOCATION, "file:///tmp/path/to/table/"); + assertThatThrownBy( + () -> catalog.createTable(identifier, newSchema, new Transform[0], newProperties)) + .isInstanceOf(TableAlreadyExistsException.class); + } + + @ParameterizedTest + @ValueSource(strings = {"delta", "csv"}) + void testCreateAndLoadGenericTable(String format) throws Exception { + String[] namespace = new String[] {"ns"}; + Identifier identifier = Identifier.of(namespace, "generic-test-table"); + Map properties = Maps.newHashMap(); + properties.put(PolarisCatalogUtils.TABLE_PROVIDER_KEY, format); + properties.put(TableCatalog.PROP_LOCATION, "file:///tmp/delta/path/to/table/"); + StructType schema = new StructType().add("boolType", "boolean"); + + assertThatThrownBy(() -> catalog.createTable(identifier, schema, new Transform[0], properties)) + .isInstanceOf(org.apache.iceberg.exceptions.NoSuchNamespaceException.class); + + catalog.createNamespace(namespace, Maps.newHashMap()); + + SQLConf conf = new SQLConf(); + try (MockedStatic mockedStaticSparkSession = + Mockito.mockStatic(SparkSession.class); + MockedStatic mockedStaticDS = Mockito.mockStatic(DataSource.class); + MockedStatic mockedStaticDSV2 = + Mockito.mockStatic(DataSourceV2Utils.class)) { + SparkSession mockedSession = Mockito.mock(SparkSession.class); + mockedStaticSparkSession.when(SparkSession::active).thenReturn(mockedSession); + SessionState mockedState = Mockito.mock(SessionState.class); + Mockito.when(mockedSession.sessionState()).thenReturn(mockedState); + Mockito.when(mockedState.conf()).thenReturn(conf); + + TableProvider provider = Mockito.mock(TableProvider.class); + mockedStaticDS + .when(() -> DataSource.lookupDataSourceV2(Mockito.eq(format), Mockito.any())) + .thenReturn(Option.apply(provider)); + V1Table table = Mockito.mock(V1Table.class); + mockedStaticDSV2 + .when( + () -> + DataSourceV2Utils.getTableFromProvider( + Mockito.eq(provider), Mockito.any(), Mockito.any())) + .thenReturn(table); + Table createdTable = catalog.createTable(identifier, schema, new Transform[0], properties); + assertThat(createdTable).isInstanceOf(V1Table.class); + + // load the table + Table loadedTable = catalog.loadTable(identifier); + assertThat(loadedTable).isInstanceOf(V1Table.class); + } + + StructType newSchema = new StructType().add("LongType", "Long"); + Map newProperties = Maps.newHashMap(); + newProperties.put(PolarisCatalogUtils.TABLE_PROVIDER_KEY, "parquet"); + newProperties.put(TableCatalog.PROP_LOCATION, "file:///tmp/path/to/table/"); + assertThatThrownBy( + () -> catalog.createTable(identifier, newSchema, new Transform[0], newProperties)) + .isInstanceOf(TableAlreadyExistsException.class); + } + @Test public void testUnsupportedOperations() { String[] namespace = new String[] {"ns1"}; From 4ab0f83649764ececce2b62b9022025bd62edbf6 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Fri, 4 Apr 2025 18:59:48 -0700 Subject: [PATCH 13/29] fix import --- plugins/spark/v3.5/build.gradle.kts | 4 +- .../polaris/spark/PolarisRESTCatalog.java | 8 +++- .../polaris/spark/PolarisInMemoryCatalog.java | 40 ++++++++----------- 3 files changed, 24 insertions(+), 28 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 66bf0dbe85..2c31e39cbd 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -103,8 +103,6 @@ tasks.register("createPolarisSparkJar") { "polaris-iceberg-${icebergVersion}-spark-runtime-${sparkMajorVersion}_${scalaVersion}" isZip64 = true - dependencies { exclude("META-INF/**") } - // pack both the source code and dependencies from(sourceSets.main.get().output) configurations = listOf(project.configurations.runtimeClasspath.get()) @@ -118,6 +116,8 @@ tasks.register("createPolarisSparkJar") { exclude(dependency("org.apache.iceberg:iceberg-spark-runtime-*.*")) exclude(dependency("org.apache.iceberg:iceberg-core*.*")) } + + relocate("com.fasterxml", "org.apache.polaris.shaded.com.fasterxml.jackson") } tasks.withType(Jar::class).named("sourcesJar") { dependsOn("createPolarisSparkJar") } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index aae7260be5..fce325e978 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.polaris.spark; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import java.io.Closeable; @@ -31,8 +32,11 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.CloseableGroup; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.rest.*; +import org.apache.iceberg.rest.Endpoint; +import org.apache.iceberg.rest.ErrorHandlers; +import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.ResourcePaths; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.util.EnvironmentUtil; diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java index 40e98a8bd1..c180edf6f9 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.polaris.spark; +import com.google.common.collect.Maps; import java.util.*; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; @@ -27,7 +28,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.inmemory.InMemoryCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.polaris.service.types.GenericTable; /** InMemory implementation for the Polaris Catalog. This class is mainly used by testing. */ @@ -70,31 +70,23 @@ public boolean dropGenericTable(TableIdentifier identifier) { @Override public GenericTable createGenericTable( TableIdentifier identifier, String format, Map props) { - synchronized (this) { - if (!namespaceExists(identifier.namespace())) { - throw new NoSuchNamespaceException( - "Cannot create generic table %s. Namespace does not exist: %s", - identifier, identifier.namespace()); - } - if (listViews(identifier.namespace()).contains(identifier)) { - throw new AlreadyExistsException("View with same name already exists: %s", identifier); - } - if (listTables(identifier.namespace()).contains(identifier)) { - throw new AlreadyExistsException( - "Iceberg table with same name already exists: %s", identifier); - } - if (this.genericTables.containsKey(identifier)) { - throw new AlreadyExistsException("Generic table %s already exists", identifier); - } - this.genericTables.compute( - identifier, - (k, table) -> { - return GenericTable.builder() - .setName(k.name()) + if (!namespaceExists(identifier.namespace())) { + throw new NoSuchNamespaceException( + "Cannot create generic table %s. Namespace does not exist: %s", + identifier, identifier.namespace()); + } + + GenericTable previous = + this.genericTables.putIfAbsent( + identifier, + GenericTable.builder() + .setName(identifier.name()) .setFormat(format) .setProperties(props) - .build(); - }); + .build()); + + if (previous != null) { + throw new AlreadyExistsException("Generic table already exists: %s", identifier); } return this.genericTables.get(identifier); From e8493d207a651036ce08abe2160d57256c6acbae Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Fri, 4 Apr 2025 19:32:55 -0700 Subject: [PATCH 14/29] add change --- plugins/spark/v3.5/build.gradle.kts | 4 ++-- .../src/main/java/org/apache/polaris/spark/SparkCatalog.java | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 2c31e39cbd..619da3f305 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -103,12 +103,12 @@ tasks.register("createPolarisSparkJar") { "polaris-iceberg-${icebergVersion}-spark-runtime-${sparkMajorVersion}_${scalaVersion}" isZip64 = true + dependencies { exclude("META-INF/**") } + // pack both the source code and dependencies from(sourceSets.main.get().output) configurations = listOf(project.configurations.runtimeClasspath.get()) - mergeServiceFiles() - // Optimization: Minimize the JAR (remove unused classes from dependencies) // The iceberg-spark-runtime plugin is always packaged along with our polaris-spark plugin, // therefore excluded from the optimization. diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index ae456a09ef..d1342a430d 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -108,6 +108,7 @@ private void initRESTCatalog(String name, CaseInsensitiveStringMap options) { @Override public void initialize(String name, CaseInsensitiveStringMap options) { + this.catalogName = name; initRESTCatalog(name, options); this.deltaHelper = new DeltaHelper(options); } From fb958d4ab94dcfa77f11b8424d9f7ba8953d4c27 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Fri, 4 Apr 2025 20:20:21 -0700 Subject: [PATCH 15/29] fix interface --- plugins/spark/v3.5/build.gradle.kts | 4 ++-- .../main/java/org/apache/polaris/spark/PolarisCatalog.java | 2 +- .../java/org/apache/polaris/spark/PolarisRESTCatalog.java | 4 ++-- .../java/org/apache/polaris/spark/PolarisSparkCatalog.java | 2 +- .../java/org/apache/polaris/spark/PolarisInMemoryCatalog.java | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 619da3f305..9b9e4ab30c 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -67,8 +67,8 @@ dependencies { ) { // exclude the iceberg rest dependencies, use the ones pulled // with iceberg-core dependency - exclude("org.apache.iceberg.rest", "*") - exclude("org.apache.iceberg.hadoop", "*") + exclude("org.apache.iceberg", "rest") + exclude("org.apache.iceberg", "hadoop") } compileOnly("org.scala-lang:scala-library:${scalaLibraryVersion}") diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java index f35d2dc83c..31a6ac1897 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisCatalog.java @@ -32,5 +32,5 @@ public interface PolarisCatalog { boolean dropGenericTable(TableIdentifier identifier); GenericTable createGenericTable( - TableIdentifier identifier, String format, Map props); + TableIdentifier identifier, String format, String doc, Map props); } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index fce325e978..593ceb00ed 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -151,10 +151,10 @@ public boolean dropGenericTable(TableIdentifier identifier) { @Override public GenericTable createGenericTable( - TableIdentifier identifier, String format, Map props) { + TableIdentifier identifier, String format, String doc, Map props) { Endpoint.check(endpoints, PolarisEndpoints.V1_CREATE_GENERIC_TABLE); CreateGenericTableRESTRequest request = - new CreateGenericTableRESTRequest(identifier.name(), format, null, props); + new CreateGenericTableRESTRequest(identifier.name(), format, doc, props); LoadGenericTableRESTResponse response = restClient diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java index 8147e197af..8f8c07fbaf 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisSparkCatalog.java @@ -80,7 +80,7 @@ public Table createTable( String format = properties.get(PolarisCatalogUtils.TABLE_PROVIDER_KEY); GenericTable genericTable = this.polarisCatalog.createGenericTable( - Spark3Util.identifierToTableIdentifier(identifier), format, properties); + Spark3Util.identifierToTableIdentifier(identifier), format, null, properties); return PolarisCatalogUtils.loadSparkTable(genericTable); } catch (AlreadyExistsException e) { throw new TableAlreadyExistsException(identifier); diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java index c180edf6f9..ea73606d59 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java @@ -69,7 +69,7 @@ public boolean dropGenericTable(TableIdentifier identifier) { @Override public GenericTable createGenericTable( - TableIdentifier identifier, String format, Map props) { + TableIdentifier identifier, String format, String doc, Map props) { if (!namespaceExists(identifier.namespace())) { throw new NoSuchNamespaceException( "Cannot create generic table %s. Namespace does not exist: %s", From 676583d889862e9ec7414d9df152e41246b92331 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Fri, 4 Apr 2025 20:27:27 -0700 Subject: [PATCH 16/29] reduce exclude --- plugins/spark/v3.5/build.gradle.kts | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 9b9e4ab30c..3734813000 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -103,7 +103,13 @@ tasks.register("createPolarisSparkJar") { "polaris-iceberg-${icebergVersion}-spark-runtime-${sparkMajorVersion}_${scalaVersion}" isZip64 = true - dependencies { exclude("META-INF/**") } + dependencies { + exclude("META-INF/*.SF") + exclude("META-INF/*.DSA") + exclude("META-INF/*.RSA") + } + + mergeServiceFiles() // pack both the source code and dependencies from(sourceSets.main.get().output) From d6aa6a00402d2bb2aa6d0f6e87a3d69dc9feaf14 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Mon, 7 Apr 2025 15:58:09 -0700 Subject: [PATCH 17/29] add invalid import check --- plugins/spark/v3.5/build.gradle.kts | 36 +++++++++++++++++++ .../polaris/spark/SparkCatalogTest.java | 14 ++------ 2 files changed, 38 insertions(+), 12 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 3734813000..b2fbb37543 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -97,6 +97,42 @@ dependencies { } } +tasks.register("checkNoDisallowedImports") { + doLast { + // List of disallowed imports (modify these according to your needs) + val disallowedImports = + listOf("import org.apache.iceberg.shaded.", "org.apache.iceberg.relocated.") + + // Directory to scan for Java files + val sourceDirs = listOf(file("src/main/java"), file("src/test/java")) + + val violations = mutableListOf() + // Scan Java files in each directory + sourceDirs.forEach { sourceDir -> + fileTree(sourceDir) + .matching { + include("**/*.java") // Only include Java files + } + .forEach { file -> + val content = file.readText() + disallowedImports.forEach { importStatement -> + if (content.contains(importStatement)) { + violations.add( + "Disallowed import found in ${file.relativeTo(projectDir)}: $importStatement" + ) + } + } + } + } + + if (violations.isNotEmpty()) { + throw GradleException("Disallowed imports found! $violations") + } + } +} + +tasks.named("check") { dependsOn("checkNoDisallowedImports") } + tasks.register("createPolarisSparkJar") { archiveClassifier = null archiveBaseName = diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index c0ec6c9e50..b4dd19115e 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -23,7 +23,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; - import com.google.common.collect.Maps; import java.util.Arrays; import java.util.Map; @@ -36,21 +35,12 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; -import org.apache.spark.sql.connector.catalog.*; -import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.source.SparkTable; import org.apache.polaris.spark.utils.DeltaHelper; import org.apache.polaris.spark.utils.PolarisCatalogUtils; -import org.apache.spark.SparkContext; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.connector.catalog.*; -import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.execution.datasources.DataSource; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; import org.apache.spark.sql.internal.SQLConf; @@ -66,7 +56,7 @@ import scala.Option; public class SparkCatalogTest { - private class InMemoryIcebergSparkCatalog extends org.apache.iceberg.spark.SparkCatalog { + private static class InMemoryIcebergSparkCatalog extends org.apache.iceberg.spark.SparkCatalog { private PolarisInMemoryCatalog inMemoryCatalog = null; @Override @@ -88,7 +78,7 @@ public PolarisInMemoryCatalog getInMemoryCatalog() { * And SparkCatalog implementation that uses InMemory catalog implementation for both Iceberg and * Polaris */ - private class InMemorySparkCatalog extends SparkCatalog { + private static class InMemorySparkCatalog extends SparkCatalog { @Override public void initialize(String name, CaseInsensitiveStringMap options) { this.catalogName = name; From 961334888aa58d7065a80e7360bcf08e5bc382c3 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Mon, 7 Apr 2025 16:44:15 -0700 Subject: [PATCH 18/29] address feedback --- plugins/spark/v3.5/build.gradle.kts | 9 --------- 1 file changed, 9 deletions(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index b2fbb37543..a9b1e721e4 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -58,9 +58,6 @@ dependencies { implementation(project(":polaris-core")) { exclude("org.apache.iceberg", "*") } implementation("org.apache.iceberg:iceberg-core:${icebergVersion}") - implementation("com.fasterxml.jackson.core:jackson-annotations") - implementation("com.fasterxml.jackson.core:jackson-core") - implementation("com.fasterxml.jackson.core:jackson-databind") implementation( "org.apache.iceberg:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}:${icebergVersion}" @@ -139,12 +136,6 @@ tasks.register("createPolarisSparkJar") { "polaris-iceberg-${icebergVersion}-spark-runtime-${sparkMajorVersion}_${scalaVersion}" isZip64 = true - dependencies { - exclude("META-INF/*.SF") - exclude("META-INF/*.DSA") - exclude("META-INF/*.RSA") - } - mergeServiceFiles() // pack both the source code and dependencies From b8a39d6cdf75984df5c24d787ddd67356ae6b7cb Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Mon, 7 Apr 2025 17:15:46 -0700 Subject: [PATCH 19/29] fix comment --- plugins/spark/v3.5/build.gradle.kts | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index a9b1e721e4..c6324b6cd0 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -96,7 +96,8 @@ dependencies { tasks.register("checkNoDisallowedImports") { doLast { - // List of disallowed imports (modify these according to your needs) + // List of disallowed imports. Right now, we disallow usage of shaded or + // relocated libraries in the iceberg spark runtime jar. val disallowedImports = listOf("import org.apache.iceberg.shaded.", "org.apache.iceberg.relocated.") From 41158db74b1f2c52881e738e05d9f44da035b0a6 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 9 Apr 2025 17:48:48 -0700 Subject: [PATCH 20/29] rebase main --- .../apache/polaris/spark/SparkCatalog.java | 21 +++---- .../polaris/spark/SparkCatalogTest.java | 57 ++++++++----------- 2 files changed, 34 insertions(+), 44 deletions(-) diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index d1342a430d..e1eb75d79a 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -19,8 +19,13 @@ package org.apache.polaris.spark; import java.util.Map; -import java.util.Set; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.spark.SupportsReplaceView; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.polaris.spark.utils.DeltaHelper; +import org.apache.polaris.spark.utils.PolarisCatalogUtils; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -38,15 +43,11 @@ import org.apache.spark.sql.connector.catalog.View; import org.apache.spark.sql.connector.catalog.ViewCatalog; import org.apache.spark.sql.connector.catalog.ViewChange; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.rest.auth.OAuth2Util; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.polaris.spark.utils.DeltaHelper; -import org.apache.polaris.spark.utils.PolarisCatalogUtils; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * SparkCatalog Implementation that is able to interact with both Iceberg SparkCatalog and Polaris @@ -62,9 +63,9 @@ public class SparkCatalog SupportsReplaceView { private static final Logger LOG = LoggerFactory.getLogger(SparkCatalog.class); - private String catalogName = null; - private org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; - private PolarisSparkCatalog polarisSparkCatalog = null; + protected String catalogName = null; + protected org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; + protected PolarisSparkCatalog polarisSparkCatalog = null; protected DeltaHelper deltaHelper = null; diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index b4dd19115e..41dfa89b36 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -19,7 +19,6 @@ package org.apache.polaris.spark; import static org.apache.iceberg.CatalogProperties.CATALOG_IMPL; -import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -28,24 +27,30 @@ import java.util.Map; import java.util.UUID; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.SparkUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.SparkContext; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; -import org.apache.spark.sql.types.StructType; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.source.SparkTable; import org.apache.polaris.spark.utils.DeltaHelper; import org.apache.polaris.spark.utils.PolarisCatalogUtils; +import org.apache.spark.SparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.NamespaceChange; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.catalog.V1Table; +import org.apache.spark.sql.connector.catalog.View; +import org.apache.spark.sql.connector.catalog.ViewChange; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.execution.datasources.DataSource; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.internal.SessionState; -import org.apache.spark.sql.types.*; +import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -100,11 +105,6 @@ public void initialize(String name, CaseInsensitiveStringMap options) { private String catalogName; private static final String[] defaultNS = new String[] {"ns"}; - private static final Schema defaultSchema = - new Schema( - 5, - required(3, "id", Types.IntegerType.get(), "unique ID"), - required(4, "data", Types.StringType.get())); @BeforeEach public void setup() throws Exception { @@ -135,7 +135,6 @@ public void setup() throws Exception { catalog.createNamespace(defaultNS, Maps.newHashMap()); } - @Test void testCreateAndLoadNamespace() throws Exception { String[] namespace = new String[] {"ns1"}; @@ -291,7 +290,7 @@ void testBasicViewOperations() throws Exception { @Test void testListViews() throws Exception { // create a new namespace under the default NS - String[] namespace = new String[]{"ns", "nsl2"}; + String[] namespace = new String[] {"ns", "nsl2"}; catalog.createNamespace(namespace, Maps.newHashMap()); // table schema StructType schema = new StructType().add("id", "long").add("name", "string"); @@ -309,10 +308,10 @@ void testListViews() throws Exception { new String[0], Maps.newHashMap()); // create two views under ns.nsl2 - String[] nsl2ViewNames = new String[]{"test-view2", "test-view3"}; + String[] nsl2ViewNames = new String[] {"test-view2", "test-view3"}; String[] nsl2ViewSQLs = - new String[]{ - "select id from test-table where id == 3", "select id from test-table where id < 3" + new String[] { + "select id from test-table where id == 3", "select id from test-table where id < 3" }; for (int i = 0; i < nsl2ViewNames.length; i++) { catalog.createView( @@ -339,19 +338,14 @@ void testListViews() throws Exception { } } + @Test void testCreateAndLoadIcebergTable() throws Exception { - String[] namespace = new String[] {"ns"}; - Identifier identifier = Identifier.of(namespace, "iceberg-table"); + Identifier identifier = Identifier.of(defaultNS, "iceberg-table"); Map properties = Maps.newHashMap(); properties.put(PolarisCatalogUtils.TABLE_PROVIDER_KEY, "iceberg"); properties.put(TableCatalog.PROP_LOCATION, "file:///tmp/path/to/table/"); StructType schema = new StructType().add("boolType", "boolean"); - assertThatThrownBy(() -> catalog.createTable(identifier, schema, new Transform[0], properties)) - .isInstanceOf(org.apache.iceberg.exceptions.NoSuchNamespaceException.class); - - catalog.createNamespace(namespace, Maps.newHashMap()); - Table createdTable = catalog.createTable(identifier, schema, new Transform[0], properties); assertThat(createdTable).isInstanceOf(SparkTable.class); @@ -373,18 +367,12 @@ void testCreateAndLoadIcebergTable() throws Exception { @ParameterizedTest @ValueSource(strings = {"delta", "csv"}) void testCreateAndLoadGenericTable(String format) throws Exception { - String[] namespace = new String[] {"ns"}; - Identifier identifier = Identifier.of(namespace, "generic-test-table"); + Identifier identifier = Identifier.of(defaultNS, "generic-test-table"); Map properties = Maps.newHashMap(); properties.put(PolarisCatalogUtils.TABLE_PROVIDER_KEY, format); properties.put(TableCatalog.PROP_LOCATION, "file:///tmp/delta/path/to/table/"); StructType schema = new StructType().add("boolType", "boolean"); - assertThatThrownBy(() -> catalog.createTable(identifier, schema, new Transform[0], properties)) - .isInstanceOf(org.apache.iceberg.exceptions.NoSuchNamespaceException.class); - - catalog.createNamespace(namespace, Maps.newHashMap()); - SQLConf conf = new SQLConf(); try (MockedStatic mockedStaticSparkSession = Mockito.mockStatic(SparkSession.class); @@ -442,5 +430,6 @@ public void testUnsupportedOperations() { assertThatThrownBy(() -> catalog.invalidateTable(identifier)) .isInstanceOf(UnsupportedOperationException.class); assertThatThrownBy(() -> catalog.purgeTable(identifier)) + .isInstanceOf(UnsupportedOperationException.class); } } From a656ed441a114733b2121d6e1fdaa0693d88c2f9 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Wed, 9 Apr 2025 22:09:02 -0700 Subject: [PATCH 21/29] add rest --- .../spark/rest/DeserializationTest.java | 88 +++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/rest/DeserializationTest.java diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/rest/DeserializationTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/rest/DeserializationTest.java new file mode 100644 index 0000000000..542fd05d8d --- /dev/null +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/rest/DeserializationTest.java @@ -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.spark.rest; + +import static org.assertj.core.api.Assertions.assertThat; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import java.util.Map; +import java.util.stream.Stream; +import org.apache.polaris.service.types.GenericTable; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +public class DeserializationTest { + private ObjectMapper mapper; + + @BeforeEach + public void setUp() { + mapper = new ObjectMapper(); + } + + @ParameterizedTest + @MethodSource("genericTableTestCases") + public void testLoadGenericTableRESTResponse(String doc, Map properties) + throws JsonProcessingException { + GenericTable table = + GenericTable.builder() + .setFormat("delta") + .setName("test-table") + .setProperties(properties) + .setDoc(doc) + .build(); + LoadGenericTableRESTResponse response = new LoadGenericTableRESTResponse(table); + String json = mapper.writeValueAsString(response); + LoadGenericTableRESTResponse deserializedResponse = + mapper.readValue(json, LoadGenericTableRESTResponse.class); + assertThat(deserializedResponse.getTable().getFormat()).isEqualTo("delta"); + assertThat(deserializedResponse.getTable().getName()).isEqualTo("test-table"); + assertThat(deserializedResponse.getTable().getDoc()).isEqualTo(doc); + assertThat(deserializedResponse.getTable().getProperties().size()).isEqualTo(properties.size()); + } + + @ParameterizedTest + @MethodSource("genericTableTestCases") + public void testCreateGenericTableRESTRequest(String doc, Map properties) + throws JsonProcessingException { + CreateGenericTableRESTRequest request = + new CreateGenericTableRESTRequest("test-table", "delta", doc, properties); + String json = mapper.writeValueAsString(request); + CreateGenericTableRESTRequest deserializedRequest = + mapper.readValue(json, CreateGenericTableRESTRequest.class); + assertThat(deserializedRequest.getName()).isEqualTo("test-table"); + assertThat(deserializedRequest.getFormat()).isEqualTo("delta"); + assertThat(deserializedRequest.getDoc()).isEqualTo(doc); + assertThat(deserializedRequest.getProperties().size()).isEqualTo(properties.size()); + } + + private static Stream genericTableTestCases() { + var doc = "table for testing"; + var properties = Maps.newHashMap(); + properties.put("location", "s3://path/to/table/"); + return Stream.of( + Arguments.of(doc, properties), + Arguments.of(null, Maps.newHashMap()), + Arguments.of(doc, Maps.newHashMap()), + Arguments.of(null, properties)); + } +} From 7ee6f6924c4472bb04dc7ccb7b812acf4838693a Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 10 Apr 2025 11:14:04 -0700 Subject: [PATCH 22/29] address feedback --- plugins/spark/v3.5/build.gradle.kts | 4 +- .../polaris/spark/utils/DeltaHelper.java | 15 ++-- .../spark/utils/PolarisCatalogUtils.java | 4 +- ...eltaCatalog.java => NoopDeltaCatalog.java} | 13 +-- .../polaris/spark/SparkCatalogTest.java | 2 +- .../core/config/FeatureConfiguration.java | 2 +- .../generic/GenericTableCatalogAdapter.java | 80 ++++++++++++++++++- 7 files changed, 101 insertions(+), 19 deletions(-) rename plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/{InMemoryDeltaCatalog.java => NoopDeltaCatalog.java} (70%) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index c6324b6cd0..839fc55658 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -64,8 +64,7 @@ dependencies { ) { // exclude the iceberg rest dependencies, use the ones pulled // with iceberg-core dependency - exclude("org.apache.iceberg", "rest") - exclude("org.apache.iceberg", "hadoop") + exclude("org.apache.iceberg", "iceberg-core") } compileOnly("org.scala-lang:scala-library:${scalaLibraryVersion}") @@ -94,6 +93,7 @@ dependencies { } } +// TODO: replace the check using gradlew checkstyle plugin tasks.register("checkNoDisallowedImports") { doLast { // List of disallowed imports. Right now, we disallow usage of shaded or diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java index 8f1d556e1d..f6b933d025 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -18,6 +18,7 @@ */ package org.apache.polaris.spark.utils; +import com.esotericsoftware.minlog.Log; import java.lang.reflect.Field; import java.lang.reflect.Method; import org.apache.iceberg.common.DynConstructors; @@ -71,18 +72,22 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { // set the polaris spark catalog as the delegate catalog of delta catalog ((DelegatingCatalogExtension) this.deltaCatalog).setDelegateCatalog(polarisSparkCatalog); - // https://github.com/delta-io/delta/issues/4306 + // We want to behave exactly the same as unity catalog for Delta. However, DeltaCatalog + // implementation today is hard coded for unity catalog. Following issue is used to track + // the extension of the usage https://github.com/delta-io/delta/issues/4306. + // Here, we use reflection to set the isUnityCatalog to true for exactly same behavior as + // unity catalog for now. try { - // Access the lazy val field's underlying method + // isUnityCatalog is a lazy val, access the compute method for the lazy val + // make sure the method is triggered before the value is set, otherwise, the + // value will be overwritten later when the method is triggered. String methodGetName = "isUnityCatalog" + "$lzycompute"; Method method = this.deltaCatalog.getClass().getDeclaredMethod(methodGetName); method.setAccessible(true); // invoke the lazy methods before it is set method.invoke(this.deltaCatalog); } catch (NoSuchMethodException e) { - throw new RuntimeException( - "Failed to find lazy compute method for isUnityCatalog, delta-spark version >= 3.2.1 is required", - e); + Log.warn("No lazy compute method found for variable isUnityCatalog"); } catch (Exception e) { throw new RuntimeException("Failed to invoke the lazy compute methods for isUnityCatalog", e); } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index 44dda0ca83..3040fd62bc 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -36,7 +36,6 @@ import org.apache.spark.sql.connector.catalog.TableProvider; import org.apache.spark.sql.execution.datasources.DataSource; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class PolarisCatalogUtils { @@ -82,9 +81,8 @@ public static Table loadSparkTable(GenericTable genericTable) { // property as path to make sure the table can be loaded. tableProperties.put(TABLE_PATH_KEY, properties.get(TableCatalog.PROP_LOCATION)); } - CaseInsensitiveStringMap property_map = new CaseInsensitiveStringMap(tableProperties); return DataSourceV2Utils.getTableFromProvider( - provider, property_map, scala.Option$.MODULE$.empty()); + provider, new CaseInsensitiveStringMap(tableProperties), scala.Option.empty()); } /** diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java similarity index 70% rename from plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java rename to plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java index 7080812808..edd7bc1341 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/InMemoryDeltaCatalog.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java @@ -20,10 +20,13 @@ import org.apache.spark.sql.connector.catalog.DelegatingCatalogExtension; -public class InMemoryDeltaCatalog extends DelegatingCatalogExtension { +/** + * This is a fake delta catalog class that is used for testing. This class is a noop class that + * directly passes all calls to the delegate CatalogPlugin configured as part of + * DelegatingCatalogExtension. + */ +public class NoopDeltaCatalog extends DelegatingCatalogExtension { + // This is a mock of isUnityCatalog scala lazy val in + // org.apache.spark.sql.delta.catalog.DeltaCatalog. private boolean isUnityCatalog = false; - - private void isUnityCatalog$lzycompute() { - isUnityCatalog = true; - } } diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index 41dfa89b36..644726c1e2 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -113,7 +113,7 @@ public void setup() throws Exception { catalogConfig.put(CATALOG_IMPL, "org.apache.iceberg.inmemory.InMemoryCatalog"); catalogConfig.put("cache-enabled", "false"); catalogConfig.put( - DeltaHelper.DELTA_CATALOG_IMPL_KEY, "org.apache.polaris.spark.InMemoryDeltaCatalog"); + DeltaHelper.DELTA_CATALOG_IMPL_KEY, "org.apache.polaris.spark.NoopDeltaCatalog"); catalog = new InMemorySparkCatalog(); Configuration conf = new Configuration(); try (MockedStatic mockedStaticSparkSession = diff --git a/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java b/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java index cc8bae454d..8e71d049c2 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java @@ -188,6 +188,6 @@ protected FeatureConfiguration( PolarisConfiguration.builder() .key("ENABLE_GENERIC_TABLES") .description("If true, the generic-tables endpoints are enabled") - .defaultValue(false) + .defaultValue(true) .buildFeatureConfiguration(); } diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java b/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java index cd8c2b5a67..8450c02f54 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java @@ -19,14 +19,78 @@ package org.apache.polaris.service.catalog.generic; import jakarta.enterprise.context.RequestScoped; +import jakarta.inject.Inject; import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.SecurityContext; +import java.net.URLEncoder; +import java.nio.charset.Charset; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NotAuthorizedException; +import org.apache.iceberg.rest.RESTUtil; +import org.apache.polaris.core.auth.AuthenticatedPolarisPrincipal; +import org.apache.polaris.core.auth.PolarisAuthorizer; +import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.persistence.PolarisEntityManager; +import org.apache.polaris.core.persistence.PolarisMetaStoreManager; +import org.apache.polaris.service.catalog.CatalogPrefixParser; import org.apache.polaris.service.catalog.api.PolarisCatalogGenericTableApiService; +import org.apache.polaris.service.context.CallContextCatalogFactory; import org.apache.polaris.service.types.CreateGenericTableRequest; @RequestScoped public class GenericTableCatalogAdapter implements PolarisCatalogGenericTableApiService { + private final RealmContext realmContext; + private final CallContext callContext; + private final CallContextCatalogFactory catalogFactory; + private final PolarisEntityManager entityManager; + private final PolarisMetaStoreManager metaStoreManager; + private final PolarisAuthorizer polarisAuthorizer; + private final CatalogPrefixParser prefixParser; + + @Inject + public GenericTableCatalogAdapter( + RealmContext realmContext, + CallContext callContext, + CallContextCatalogFactory catalogFactory, + PolarisEntityManager entityManager, + PolarisMetaStoreManager metaStoreManager, + PolarisAuthorizer polarisAuthorizer, + CatalogPrefixParser prefixParser) { + this.realmContext = realmContext; + this.callContext = callContext; + this.catalogFactory = catalogFactory; + this.entityManager = entityManager; + this.metaStoreManager = metaStoreManager; + this.polarisAuthorizer = polarisAuthorizer; + this.prefixParser = prefixParser; + + // FIXME: This is a hack to set the current context for downstream calls. + CallContext.setCurrentContext(callContext); + } + + private GenericTableCatalogHandler newHandlerWrapper( + SecurityContext securityContext, String catalogName) { + AuthenticatedPolarisPrincipal authenticatedPrincipal = + (AuthenticatedPolarisPrincipal) securityContext.getUserPrincipal(); + if (authenticatedPrincipal == null) { + throw new NotAuthorizedException("Failed to find authenticatedPrincipal in SecurityContext"); + } + + return new GenericTableCatalogHandler( + callContext, + entityManager, + metaStoreManager, + securityContext, + catalogName, + polarisAuthorizer); + } + + private static Namespace decodeNamespace(String namespace) { + return RESTUtil.decodeNamespace(URLEncoder.encode(namespace, Charset.defaultCharset())); + } + @Override public Response createGenericTable( String prefix, @@ -34,7 +98,16 @@ public Response createGenericTable( CreateGenericTableRequest createGenericTableRequest, RealmContext realmContext, SecurityContext securityContext) { - return Response.status(501).build(); // not implemented + Namespace ns = decodeNamespace(namespace); + + return Response.ok( + newHandlerWrapper(securityContext, prefix) + .createGenericTable( + TableIdentifier.of(ns, createGenericTableRequest.getName()), + createGenericTableRequest.getFormat(), + createGenericTableRequest.getDoc(), + createGenericTableRequest.getProperties())) + .build(); } @Override @@ -65,6 +138,9 @@ public Response loadGenericTable( String genericTable, RealmContext realmContext, SecurityContext securityContext) { - return Response.status(501).build(); // not implemented + Namespace ns = decodeNamespace(namespace); + TableIdentifier tableIdentifier = TableIdentifier.of(ns, RESTUtil.decodeString(genericTable)); + return Response.ok(newHandlerWrapper(securityContext, prefix).loadGenericTable(tableIdentifier)) + .build(); } } From 625c019979a1ab903975c1207a0714f63f3a5741 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 10 Apr 2025 11:14:11 -0700 Subject: [PATCH 23/29] Revert "Generalize bootstrapping in servers (#1313)" This reverts commit cd1ebb51638f9ac4244d848924fcdcd812748f81. --- .../LocalPolarisMetaStoreManagerFactory.java | 5 ----- .../service/quarkus/config/QuarkusProducers.java | 10 ++++------ .../persistence/QuarkusPersistenceConfiguration.java | 9 --------- 3 files changed, 4 insertions(+), 20 deletions(-) diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java index 7350b6b536..429cbb5eb4 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java @@ -118,11 +118,6 @@ public synchronized Map bootstrapRealms( PrincipalSecretsResult secretsResult = bootstrapServiceAndCreatePolarisPrincipalForRealm( realmContext, metaStoreManagerMap.get(realmContext.getRealmIdentifier())); - - if (rootCredentialsSet.credentials().containsKey(realm)) { - LOGGER.info("Bootstrapped realm {} using preset credentials.", realm); - } - results.put(realmContext.getRealmIdentifier(), secretsResult); } } diff --git a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java index ffab9a9412..16b7439791 100644 --- a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java +++ b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java @@ -45,7 +45,6 @@ import org.apache.polaris.core.persistence.MetaStoreManagerFactory; import org.apache.polaris.core.persistence.PolarisEntityManager; import org.apache.polaris.core.persistence.PolarisMetaStoreManager; -import org.apache.polaris.core.persistence.bootstrap.RootCredentialsSet; import org.apache.polaris.core.storage.cache.StorageCredentialCache; import org.apache.polaris.service.auth.ActiveRolesProvider; import org.apache.polaris.service.auth.Authenticator; @@ -56,6 +55,7 @@ import org.apache.polaris.service.context.RealmContextConfiguration; import org.apache.polaris.service.context.RealmContextFilter; import org.apache.polaris.service.context.RealmContextResolver; +import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory; import org.apache.polaris.service.quarkus.auth.QuarkusAuthenticationConfiguration; import org.apache.polaris.service.quarkus.catalog.io.QuarkusFileIOConfiguration; import org.apache.polaris.service.quarkus.context.QuarkusRealmContextConfiguration; @@ -154,14 +154,12 @@ public MetaStoreManagerFactory metaStoreManagerFactory( * Eagerly initialize the in-memory default realm on startup, so that users can check the * credentials printed to stdout immediately. */ - public void maybeBootstrap( + public void maybeInitializeInMemoryRealm( @Observes StartupEvent event, MetaStoreManagerFactory factory, - QuarkusPersistenceConfiguration config, RealmContextConfiguration realmContextConfiguration) { - if (config.isAutoBootstrap()) { - RootCredentialsSet rootCredentialsSet = RootCredentialsSet.fromEnvironment(); - factory.bootstrapRealms(realmContextConfiguration.realms(), rootCredentialsSet); + if (factory instanceof InMemoryPolarisMetaStoreManagerFactory) { + ((InMemoryPolarisMetaStoreManagerFactory) factory).onStartup(realmContextConfiguration); } } diff --git a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java index eb4ab6aa6e..e6306d8c7f 100644 --- a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java +++ b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java @@ -19,8 +19,6 @@ package org.apache.polaris.service.quarkus.persistence; import io.smallrye.config.ConfigMapping; -import io.smallrye.config.WithDefault; -import java.util.Set; @ConfigMapping(prefix = "polaris.persistence") public interface QuarkusPersistenceConfiguration { @@ -30,11 +28,4 @@ public interface QuarkusPersistenceConfiguration { * org.apache.polaris.core.persistence.MetaStoreManagerFactory} identifier. */ String type(); - - @WithDefault("in-memory") - Set autoBootstrapTypes(); - - default boolean isAutoBootstrap() { - return autoBootstrapTypes().contains(type()); - } } From 406c71074bd05b20f68521924d686fe2fe0f4ccf Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 10 Apr 2025 11:49:15 -0700 Subject: [PATCH 24/29] revert chagnes --- .../core/config/FeatureConfiguration.java | 2 +- .../generic/GenericTableCatalogAdapter.java | 80 +------------------ 2 files changed, 3 insertions(+), 79 deletions(-) diff --git a/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java b/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java index 8e71d049c2..cc8bae454d 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/config/FeatureConfiguration.java @@ -188,6 +188,6 @@ protected FeatureConfiguration( PolarisConfiguration.builder() .key("ENABLE_GENERIC_TABLES") .description("If true, the generic-tables endpoints are enabled") - .defaultValue(true) + .defaultValue(false) .buildFeatureConfiguration(); } diff --git a/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java b/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java index 8450c02f54..cd8c2b5a67 100644 --- a/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java +++ b/service/common/src/main/java/org/apache/polaris/service/catalog/generic/GenericTableCatalogAdapter.java @@ -19,78 +19,14 @@ package org.apache.polaris.service.catalog.generic; import jakarta.enterprise.context.RequestScoped; -import jakarta.inject.Inject; import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.SecurityContext; -import java.net.URLEncoder; -import java.nio.charset.Charset; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NotAuthorizedException; -import org.apache.iceberg.rest.RESTUtil; -import org.apache.polaris.core.auth.AuthenticatedPolarisPrincipal; -import org.apache.polaris.core.auth.PolarisAuthorizer; -import org.apache.polaris.core.context.CallContext; import org.apache.polaris.core.context.RealmContext; -import org.apache.polaris.core.persistence.PolarisEntityManager; -import org.apache.polaris.core.persistence.PolarisMetaStoreManager; -import org.apache.polaris.service.catalog.CatalogPrefixParser; import org.apache.polaris.service.catalog.api.PolarisCatalogGenericTableApiService; -import org.apache.polaris.service.context.CallContextCatalogFactory; import org.apache.polaris.service.types.CreateGenericTableRequest; @RequestScoped public class GenericTableCatalogAdapter implements PolarisCatalogGenericTableApiService { - private final RealmContext realmContext; - private final CallContext callContext; - private final CallContextCatalogFactory catalogFactory; - private final PolarisEntityManager entityManager; - private final PolarisMetaStoreManager metaStoreManager; - private final PolarisAuthorizer polarisAuthorizer; - private final CatalogPrefixParser prefixParser; - - @Inject - public GenericTableCatalogAdapter( - RealmContext realmContext, - CallContext callContext, - CallContextCatalogFactory catalogFactory, - PolarisEntityManager entityManager, - PolarisMetaStoreManager metaStoreManager, - PolarisAuthorizer polarisAuthorizer, - CatalogPrefixParser prefixParser) { - this.realmContext = realmContext; - this.callContext = callContext; - this.catalogFactory = catalogFactory; - this.entityManager = entityManager; - this.metaStoreManager = metaStoreManager; - this.polarisAuthorizer = polarisAuthorizer; - this.prefixParser = prefixParser; - - // FIXME: This is a hack to set the current context for downstream calls. - CallContext.setCurrentContext(callContext); - } - - private GenericTableCatalogHandler newHandlerWrapper( - SecurityContext securityContext, String catalogName) { - AuthenticatedPolarisPrincipal authenticatedPrincipal = - (AuthenticatedPolarisPrincipal) securityContext.getUserPrincipal(); - if (authenticatedPrincipal == null) { - throw new NotAuthorizedException("Failed to find authenticatedPrincipal in SecurityContext"); - } - - return new GenericTableCatalogHandler( - callContext, - entityManager, - metaStoreManager, - securityContext, - catalogName, - polarisAuthorizer); - } - - private static Namespace decodeNamespace(String namespace) { - return RESTUtil.decodeNamespace(URLEncoder.encode(namespace, Charset.defaultCharset())); - } - @Override public Response createGenericTable( String prefix, @@ -98,16 +34,7 @@ public Response createGenericTable( CreateGenericTableRequest createGenericTableRequest, RealmContext realmContext, SecurityContext securityContext) { - Namespace ns = decodeNamespace(namespace); - - return Response.ok( - newHandlerWrapper(securityContext, prefix) - .createGenericTable( - TableIdentifier.of(ns, createGenericTableRequest.getName()), - createGenericTableRequest.getFormat(), - createGenericTableRequest.getDoc(), - createGenericTableRequest.getProperties())) - .build(); + return Response.status(501).build(); // not implemented } @Override @@ -138,9 +65,6 @@ public Response loadGenericTable( String genericTable, RealmContext realmContext, SecurityContext securityContext) { - Namespace ns = decodeNamespace(namespace); - TableIdentifier tableIdentifier = TableIdentifier.of(ns, RESTUtil.decodeString(genericTable)); - return Response.ok(newHandlerWrapper(securityContext, prefix).loadGenericTable(tableIdentifier)) - .build(); + return Response.status(501).build(); // not implemented } } From 2131b89550b42c61955e2e2ce96e5126d9a70971 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 10 Apr 2025 11:56:42 -0700 Subject: [PATCH 25/29] update comment --- .../apache/polaris/spark/PolarisInMemoryCatalog.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java index ea73606d59..5c3d597100 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java @@ -19,7 +19,9 @@ package org.apache.polaris.spark; import com.google.common.collect.Maps; -import java.util.*; +import java.util.Comparator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; import org.apache.iceberg.catalog.Namespace; @@ -58,13 +60,7 @@ public GenericTable loadGenericTable(TableIdentifier identifier) { @Override public boolean dropGenericTable(TableIdentifier identifier) { - synchronized (this) { - if (null == this.genericTables.remove(identifier)) { - return false; - } - } - - return true; + return null != this.genericTables.remove(identifier); } @Override From cf0e68e9ac4cc099575ba4a1a96f5a5bdf694c0b Mon Sep 17 00:00:00 2001 From: Dmitri Bourlatchkov Date: Tue, 8 Apr 2025 13:41:57 -0400 Subject: [PATCH 26/29] Generalize bootstrapping in servers (#1313) * Remove `instanceof` checks from `QuarkusProducers`. * Remove the now unused `onStartup` method from `InMemoryPolarisMetaStoreManagerFactory`. * Instead, call the good old `bootstrapRealms` method from `QuarkusProducers`. * Add new config property to control which MetaStore types are bootstrapped automatically (defaults to `in-memory` as before). * There is no bootstrap behaviour change in this PR, only refactorings to simplify code. * Add info log message to indicate when a realm is bootstrapped in runtime using preset credentials. Future enhancements may include pulling preset credentials from a secret manager like Vault for bootstrapping (s discussed in comments on #1228). --- .../LocalPolarisMetaStoreManagerFactory.java | 5 +++++ .../service/quarkus/config/QuarkusProducers.java | 10 ++++++---- .../persistence/QuarkusPersistenceConfiguration.java | 9 +++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java b/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java index 429cbb5eb4..7350b6b536 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/persistence/LocalPolarisMetaStoreManagerFactory.java @@ -118,6 +118,11 @@ public synchronized Map bootstrapRealms( PrincipalSecretsResult secretsResult = bootstrapServiceAndCreatePolarisPrincipalForRealm( realmContext, metaStoreManagerMap.get(realmContext.getRealmIdentifier())); + + if (rootCredentialsSet.credentials().containsKey(realm)) { + LOGGER.info("Bootstrapped realm {} using preset credentials.", realm); + } + results.put(realmContext.getRealmIdentifier(), secretsResult); } } diff --git a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java index 16b7439791..ffab9a9412 100644 --- a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java +++ b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/config/QuarkusProducers.java @@ -45,6 +45,7 @@ import org.apache.polaris.core.persistence.MetaStoreManagerFactory; import org.apache.polaris.core.persistence.PolarisEntityManager; import org.apache.polaris.core.persistence.PolarisMetaStoreManager; +import org.apache.polaris.core.persistence.bootstrap.RootCredentialsSet; import org.apache.polaris.core.storage.cache.StorageCredentialCache; import org.apache.polaris.service.auth.ActiveRolesProvider; import org.apache.polaris.service.auth.Authenticator; @@ -55,7 +56,6 @@ import org.apache.polaris.service.context.RealmContextConfiguration; import org.apache.polaris.service.context.RealmContextFilter; import org.apache.polaris.service.context.RealmContextResolver; -import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory; import org.apache.polaris.service.quarkus.auth.QuarkusAuthenticationConfiguration; import org.apache.polaris.service.quarkus.catalog.io.QuarkusFileIOConfiguration; import org.apache.polaris.service.quarkus.context.QuarkusRealmContextConfiguration; @@ -154,12 +154,14 @@ public MetaStoreManagerFactory metaStoreManagerFactory( * Eagerly initialize the in-memory default realm on startup, so that users can check the * credentials printed to stdout immediately. */ - public void maybeInitializeInMemoryRealm( + public void maybeBootstrap( @Observes StartupEvent event, MetaStoreManagerFactory factory, + QuarkusPersistenceConfiguration config, RealmContextConfiguration realmContextConfiguration) { - if (factory instanceof InMemoryPolarisMetaStoreManagerFactory) { - ((InMemoryPolarisMetaStoreManagerFactory) factory).onStartup(realmContextConfiguration); + if (config.isAutoBootstrap()) { + RootCredentialsSet rootCredentialsSet = RootCredentialsSet.fromEnvironment(); + factory.bootstrapRealms(realmContextConfiguration.realms(), rootCredentialsSet); } } diff --git a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java index e6306d8c7f..eb4ab6aa6e 100644 --- a/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java +++ b/quarkus/service/src/main/java/org/apache/polaris/service/quarkus/persistence/QuarkusPersistenceConfiguration.java @@ -19,6 +19,8 @@ package org.apache.polaris.service.quarkus.persistence; import io.smallrye.config.ConfigMapping; +import io.smallrye.config.WithDefault; +import java.util.Set; @ConfigMapping(prefix = "polaris.persistence") public interface QuarkusPersistenceConfiguration { @@ -28,4 +30,11 @@ public interface QuarkusPersistenceConfiguration { * org.apache.polaris.core.persistence.MetaStoreManagerFactory} identifier. */ String type(); + + @WithDefault("in-memory") + Set autoBootstrapTypes(); + + default boolean isAutoBootstrap() { + return autoBootstrapTypes().contains(type()); + } } From 9f48675c5d0da73060346a6164bdfa2ef5038978 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 10 Apr 2025 16:15:48 -0700 Subject: [PATCH 27/29] add scala reflect --- plugins/spark/v3.5/build.gradle.kts | 1 + .../test/java/org/apache/polaris/spark/NoopDeltaCatalog.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/plugins/spark/v3.5/build.gradle.kts b/plugins/spark/v3.5/build.gradle.kts index 839fc55658..df37fa229e 100644 --- a/plugins/spark/v3.5/build.gradle.kts +++ b/plugins/spark/v3.5/build.gradle.kts @@ -68,6 +68,7 @@ dependencies { } compileOnly("org.scala-lang:scala-library:${scalaLibraryVersion}") + compileOnly("org.scala-lang:scala-reflect:${scalaLibraryVersion}") compileOnly("org.apache.spark:spark-sql_${scalaVersion}:${spark35Version}") { // exclude log4j dependencies exclude("org.apache.logging.log4j", "log4j-slf4j2-impl") diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java index edd7bc1341..c11e8de3bf 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/NoopDeltaCatalog.java @@ -26,7 +26,7 @@ * DelegatingCatalogExtension. */ public class NoopDeltaCatalog extends DelegatingCatalogExtension { - // This is a mock of isUnityCatalog scala lazy val in + // This is a mock of isUnityCatalog scala val in // org.apache.spark.sql.delta.catalog.DeltaCatalog. private boolean isUnityCatalog = false; } From 5b328423d4719d7d4676a7fce41861453fd72840 Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Thu, 10 Apr 2025 18:27:06 -0700 Subject: [PATCH 28/29] address feedback --- .../polaris/spark/PolarisRESTCatalog.java | 30 +++++------ .../apache/polaris/spark/SparkCatalog.java | 54 +++++++++++++------ .../spark/utils/PolarisCatalogUtils.java | 8 --- .../polaris/spark/SparkCatalogTest.java | 22 ++++++++ 4 files changed, 73 insertions(+), 41 deletions(-) diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java index 593ceb00ed..0b8743132e 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/PolarisRESTCatalog.java @@ -40,25 +40,25 @@ import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.util.EnvironmentUtil; -import org.apache.iceberg.util.PropertyUtil; import org.apache.polaris.core.rest.PolarisEndpoints; import org.apache.polaris.core.rest.PolarisResourcePaths; import org.apache.polaris.service.types.GenericTable; import org.apache.polaris.spark.rest.CreateGenericTableRESTRequest; import org.apache.polaris.spark.rest.LoadGenericTableRESTResponse; -import org.apache.polaris.spark.utils.PolarisCatalogUtils; +/** + * [[PolarisRESTCatalog]] talks to Polaris REST APIs, and implements the PolarisCatalog interfaces, + * which are generic table related APIs at this moment. This class doesn't interact with any Spark + * objects. + */ public class PolarisRESTCatalog implements PolarisCatalog, Closeable { - public static final String REST_PAGE_SIZE = "rest-page-size"; - private final Function, RESTClient> clientBuilder; private RESTClient restClient = null; private CloseableGroup closeables = null; private Set endpoints; private OAuth2Util.AuthSession catalogAuth = null; - private PolarisResourcePaths paths = null; - private Integer pageSize = null; + private PolarisResourcePaths pathGenerator = null; // the default endpoints to config if server doesn't specify the 'endpoints' configuration. private static final Set DEFAULT_ENDPOINTS = PolarisEndpoints.GENERIC_TABLE_ENDPOINTS; @@ -74,7 +74,10 @@ public PolarisRESTCatalog(Function, RESTClient> clientBuilde public void initialize(Map unresolved, OAuth2Util.AuthSession catalogAuth) { Preconditions.checkArgument(unresolved != null, "Invalid configuration: null"); - // resolve any configuration that is supplied by environment variables + // Resolve any configuration that is supplied by environment variables. + // For example: if we have an entity ("key", "env:envVar") in the unresolved, + // and envVar is configured to envValue in system env. After resolve, we got + // entity ("key", "envValue"). Map props = EnvironmentUtil.resolveAll(unresolved); // TODO: switch to use authManager once iceberg dependency is updated to 1.9.0 @@ -95,15 +98,9 @@ public void initialize(Map unresolved, OAuth2Util.AuthSession ca this.endpoints = ImmutableSet.copyOf(config.endpoints()); } - this.paths = PolarisResourcePaths.forCatalogProperties(mergedProps); + this.pathGenerator = PolarisResourcePaths.forCatalogProperties(mergedProps); this.restClient = clientBuilder.apply(mergedProps).withAuthSession(catalogAuth); - this.pageSize = PropertyUtil.propertyAsNullableInt(mergedProps, REST_PAGE_SIZE); - if (pageSize != null) { - Preconditions.checkArgument( - pageSize > 0, "Invalid value for %s, must be a positive integer", REST_PAGE_SIZE); - } - this.closeables = new CloseableGroup(); this.closeables.addCloseable(this.restClient); this.closeables.setSuppressCloseFailure(true); @@ -160,7 +157,7 @@ public GenericTable createGenericTable( restClient .withAuthSession(this.catalogAuth) .post( - paths.genericTables(identifier.namespace()), + pathGenerator.genericTables(identifier.namespace()), request, LoadGenericTableRESTResponse.class, Map.of(), @@ -172,12 +169,11 @@ public GenericTable createGenericTable( @Override public GenericTable loadGenericTable(TableIdentifier identifier) { Endpoint.check(endpoints, PolarisEndpoints.V1_LOAD_GENERIC_TABLE); - PolarisCatalogUtils.checkIdentifierIsValid(identifier); LoadGenericTableRESTResponse response = restClient .withAuthSession(this.catalogAuth) .get( - paths.genericTable(identifier), + pathGenerator.genericTable(identifier), null, LoadGenericTableRESTResponse.class, Map.of(), diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index e1eb75d79a..5b8c560966 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -18,7 +18,9 @@ */ package org.apache.polaris.spark; +import com.google.common.collect.Maps; import java.util.Map; +import org.apache.arrow.util.VisibleForTesting; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.rest.auth.OAuth2Util; @@ -63,11 +65,10 @@ public class SparkCatalog SupportsReplaceView { private static final Logger LOG = LoggerFactory.getLogger(SparkCatalog.class); - protected String catalogName = null; - protected org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; - protected PolarisSparkCatalog polarisSparkCatalog = null; - - protected DeltaHelper deltaHelper = null; + @VisibleForTesting protected String catalogName = null; + @VisibleForTesting protected org.apache.iceberg.spark.SparkCatalog icebergsSparkCatalog = null; + @VisibleForTesting protected PolarisSparkCatalog polarisSparkCatalog = null; + @VisibleForTesting protected DeltaHelper deltaHelper = null; @Override public String name() { @@ -75,28 +76,49 @@ public String name() { } /** - * Initialize REST Catalog for Iceberg and Polaris, this is the only catalog type supported by - * Polaris at this moment. + * Check whether invalid catalog configuration is provided, and return an option map with catalog + * type configured correctly. This function mainly validates two parts: 1) No customized catalog + * implementation is provided. 2) No non-rest catalog type is configured. */ - private void initRESTCatalog(String name, CaseInsensitiveStringMap options) { - // TODO: relax this in the future + private CaseInsensitiveStringMap validateAndResolveCatalogOptions( + CaseInsensitiveStringMap options) { String catalogType = PropertyUtil.propertyAsString( options, CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); - if (!catalogType.equals(CatalogUtil.ICEBERG_CATALOG_TYPE_REST)) { - throw new UnsupportedOperationException( - "Only rest catalog type is supported, but got catalog type: " + catalogType); + if (catalogType != null && !catalogType.equals(CatalogUtil.ICEBERG_CATALOG_TYPE_REST)) { + throw new IllegalStateException( + "Only rest catalog type is allowed, but got catalog type: " + + catalogType + + ". Either configure the type to rest or remove the config"); } String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); if (catalogImpl != null) { - throw new UnsupportedOperationException( - "Customized catalog implementation is currently not supported!"); + throw new IllegalStateException( + "Customized catalog implementation is not supported and not needed, please remove the configuration!"); } + Map resolvedOptions = Maps.newHashMap(); + resolvedOptions.putAll(options); + if (catalogType == null) { + // if no catalog type is provided, set the catalog type to rest to ensure iceberg + // spark Catalog can be started correctly. + resolvedOptions.put(CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); + } + + return new CaseInsensitiveStringMap(resolvedOptions); + } + + /** + * Initialize REST Catalog for Iceberg and Polaris, this is the only catalog type supported by + * Polaris at this moment. + */ + private void initRESTCatalog(String name, CaseInsensitiveStringMap options) { + CaseInsensitiveStringMap resolvedOptions = validateAndResolveCatalogOptions(options); + // initialize the icebergSparkCatalog this.icebergsSparkCatalog = new org.apache.iceberg.spark.SparkCatalog(); - this.icebergsSparkCatalog.initialize(name, options); + this.icebergsSparkCatalog.initialize(name, resolvedOptions); // initialize the polaris spark catalog OAuth2Util.AuthSession catalogAuth = @@ -104,7 +126,7 @@ private void initRESTCatalog(String name, CaseInsensitiveStringMap options) { PolarisRESTCatalog restCatalog = new PolarisRESTCatalog(); restCatalog.initialize(options, catalogAuth); this.polarisSparkCatalog = new PolarisSparkCatalog(restCatalog); - this.polarisSparkCatalog.initialize(name, options); + this.polarisSparkCatalog.initialize(name, resolvedOptions); } @Override diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java index 3040fd62bc..01a4af45da 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java @@ -23,8 +23,6 @@ import java.util.Map; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.rest.RESTCatalog; import org.apache.iceberg.rest.RESTSessionCatalog; import org.apache.iceberg.rest.auth.OAuth2Util; @@ -42,12 +40,6 @@ public class PolarisCatalogUtils { public static final String TABLE_PROVIDER_KEY = "provider"; public static final String TABLE_PATH_KEY = "path"; - public static void checkIdentifierIsValid(TableIdentifier tableIdentifier) { - if (tableIdentifier.namespace().isEmpty()) { - throw new NoSuchTableException("Invalid table identifier: %s", tableIdentifier); - } - } - /** Check whether the table provider is iceberg. */ public static boolean useIceberg(String provider) { return provider == null || "iceberg".equalsIgnoreCase(provider); diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index 644726c1e2..ef699fcee2 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.UUID; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.spark.source.SparkTable; @@ -135,6 +136,27 @@ public void setup() throws Exception { catalog.createNamespace(defaultNS, Maps.newHashMap()); } + @Test + void testInvalidCatalogOptions() { + Map catalogConfigWithImpl = Maps.newHashMap(); + catalogConfigWithImpl.put(CATALOG_IMPL, "org.apache.iceberg.inmemory.InMemoryCatalog"); + catalogConfigWithImpl.put("cache-enabled", "false"); + SparkCatalog testCatalog = new SparkCatalog(); + assertThatThrownBy( + () -> + testCatalog.initialize("test", new CaseInsensitiveStringMap(catalogConfigWithImpl))) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Customized catalog implementation is not supported and not needed"); + + Map catalogConfigInvalidType = Maps.newHashMap(); + catalogConfigInvalidType.put(CatalogUtil.ICEBERG_CATALOG_TYPE, "hive"); + assertThatThrownBy( + () -> + testCatalog.initialize( + "test", new CaseInsensitiveStringMap(catalogConfigInvalidType))) + .isInstanceOf(IllegalStateException.class); + } + @Test void testCreateAndLoadNamespace() throws Exception { String[] namespace = new String[] {"ns1"}; From b3c7f9a46801c2aba6bc2384954bb68f2b347f5f Mon Sep 17 00:00:00 2001 From: Yun Zou Date: Fri, 11 Apr 2025 10:12:58 -0700 Subject: [PATCH 29/29] address comments --- .../apache/polaris/spark/SparkCatalog.java | 33 +++++++++---------- .../polaris/spark/utils/DeltaHelper.java | 3 +- .../polaris/spark/SparkCatalogTest.java | 19 +++++++---- 3 files changed, 29 insertions(+), 26 deletions(-) diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java index 5b8c560966..cf46d9a154 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/SparkCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.polaris.spark; +import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import java.util.Map; import org.apache.arrow.util.VisibleForTesting; @@ -80,31 +81,27 @@ public String name() { * type configured correctly. This function mainly validates two parts: 1) No customized catalog * implementation is provided. 2) No non-rest catalog type is configured. */ - private CaseInsensitiveStringMap validateAndResolveCatalogOptions( + @VisibleForTesting + public CaseInsensitiveStringMap validateAndResolveCatalogOptions( CaseInsensitiveStringMap options) { + Preconditions.checkArgument( + options.get(CatalogProperties.CATALOG_IMPL) == null, + "Customized catalog implementation is not supported and not needed, please remove the configuration!"); + String catalogType = PropertyUtil.propertyAsString( options, CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); - if (catalogType != null && !catalogType.equals(CatalogUtil.ICEBERG_CATALOG_TYPE_REST)) { - throw new IllegalStateException( - "Only rest catalog type is allowed, but got catalog type: " - + catalogType - + ". Either configure the type to rest or remove the config"); - } - - String catalogImpl = options.get(CatalogProperties.CATALOG_IMPL); - if (catalogImpl != null) { - throw new IllegalStateException( - "Customized catalog implementation is not supported and not needed, please remove the configuration!"); - } + Preconditions.checkArgument( + catalogType.equals(CatalogUtil.ICEBERG_CATALOG_TYPE_REST), + "Only rest catalog type is allowed, but got catalog type: " + + catalogType + + ". Either configure the type to rest or remove the config"); Map resolvedOptions = Maps.newHashMap(); resolvedOptions.putAll(options); - if (catalogType == null) { - // if no catalog type is provided, set the catalog type to rest to ensure iceberg - // spark Catalog can be started correctly. - resolvedOptions.put(CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); - } + // when no catalog type is configured, iceberg uses hive by default. Here, we make sure the + // type is set to rest since we only support rest catalog. + resolvedOptions.put(CatalogUtil.ICEBERG_CATALOG_TYPE, CatalogUtil.ICEBERG_CATALOG_TYPE_REST); return new CaseInsensitiveStringMap(resolvedOptions); } diff --git a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java index f6b933d025..2974384247 100644 --- a/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java +++ b/plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/DeltaHelper.java @@ -18,7 +18,6 @@ */ package org.apache.polaris.spark.utils; -import com.esotericsoftware.minlog.Log; import java.lang.reflect.Field; import java.lang.reflect.Method; import org.apache.iceberg.common.DynConstructors; @@ -87,7 +86,7 @@ public TableCatalog loadDeltaCatalog(PolarisSparkCatalog polarisSparkCatalog) { // invoke the lazy methods before it is set method.invoke(this.deltaCatalog); } catch (NoSuchMethodException e) { - Log.warn("No lazy compute method found for variable isUnityCatalog"); + LOG.warn("No lazy compute method found for variable isUnityCatalog"); } catch (Exception e) { throw new RuntimeException("Failed to invoke the lazy compute methods for isUnityCatalog", e); } diff --git a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java index ef699fcee2..0d142cbcbe 100644 --- a/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java +++ b/plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/SparkCatalogTest.java @@ -137,24 +137,31 @@ public void setup() throws Exception { } @Test - void testInvalidCatalogOptions() { + void testCatalogValidation() { Map catalogConfigWithImpl = Maps.newHashMap(); catalogConfigWithImpl.put(CATALOG_IMPL, "org.apache.iceberg.inmemory.InMemoryCatalog"); catalogConfigWithImpl.put("cache-enabled", "false"); SparkCatalog testCatalog = new SparkCatalog(); assertThatThrownBy( () -> - testCatalog.initialize("test", new CaseInsensitiveStringMap(catalogConfigWithImpl))) - .isInstanceOf(IllegalStateException.class) + testCatalog.validateAndResolveCatalogOptions( + new CaseInsensitiveStringMap(catalogConfigWithImpl))) + .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Customized catalog implementation is not supported and not needed"); Map catalogConfigInvalidType = Maps.newHashMap(); catalogConfigInvalidType.put(CatalogUtil.ICEBERG_CATALOG_TYPE, "hive"); assertThatThrownBy( () -> - testCatalog.initialize( - "test", new CaseInsensitiveStringMap(catalogConfigInvalidType))) - .isInstanceOf(IllegalStateException.class); + testCatalog.validateAndResolveCatalogOptions( + new CaseInsensitiveStringMap(catalogConfigInvalidType))) + .isInstanceOf(IllegalArgumentException.class); + + CaseInsensitiveStringMap resolvedMap = + testCatalog.validateAndResolveCatalogOptions( + new CaseInsensitiveStringMap(Maps.newHashMap())); + assertThat(resolvedMap.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .isEqualTo(CatalogUtil.ICEBERG_CATALOG_TYPE_REST); } @Test