gh-yzou commented on code in PR #1303:
URL: https://github.com/apache/polaris/pull/1303#discussion_r2038116609


##########
plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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 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.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;
+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;
+
+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);
+  }
+
+  /** 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<String, String> properties = genericTable.getProperties();
+    boolean hasLocationClause = properties.get(TableCatalog.PROP_LOCATION) != 
null;
+    boolean hasPathClause = properties.get(TABLE_PATH_KEY) != null;
+    Map<String, String> tableProperties = Maps.newHashMap();
+    tableProperties.putAll(properties);
+    if (!hasPathClause && hasLocationClause) {
+      // 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);

Review Comment:
   I don't think i need a separate variable here, simply inlined the definition



##########
plugins/spark/v3.5/src/test/java/org/apache/polaris/spark/PolarisInMemoryCatalog.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.Maps;
+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.NoSuchNamespaceException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.inmemory.InMemoryCatalog;
+import org.apache.polaris.service.types.GenericTable;
+
+/** InMemory implementation for the Polaris Catalog. This class is mainly used 
by testing. */
+public class PolarisInMemoryCatalog extends InMemoryCatalog implements 
PolarisCatalog {
+  private final ConcurrentMap<TableIdentifier, GenericTable> genericTables;
+
+  public PolarisInMemoryCatalog() {
+    this.genericTables = Maps.newConcurrentMap();
+  }
+
+  @Override
+  public List<TableIdentifier> 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;
+      }
+    }

Review Comment:
   updated



##########
plugins/spark/v3.5/build.gradle.kts:
##########
@@ -78,24 +94,64 @@ dependencies {
   }
 }
 
+tasks.register("checkNoDisallowedImports") {

Review Comment:
   I need to spend some to look into use the checkstyle, i added a TODO here 
for now and will follow it up in a different PR



##########
plugins/spark/v3.5/build.gradle.kts:
##########
@@ -41,18 +41,34 @@ val scalaVersion = getAndUseScalaVersionForProject()
 val icebergVersion = pluginlibs.versions.iceberg.get()
 val spark35Version = pluginlibs.versions.spark35.get()
 
+val scalaLibraryVersion =
+  if (scalaVersion == "2.12") {
+    pluginlibs.versions.scala212.get()
+  } else {
+    pluginlibs.versions.scala213.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-core:${icebergVersion}")
 
   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")

Review Comment:
   Sorry, i might confusing my self here about the modulars, i want to exclude 
the whole iceberg-core, which is pulled independently. Updated



##########
plugins/spark/v3.5/src/main/java/org/apache/polaris/spark/utils/PolarisCatalogUtils.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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 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.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;
+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;
+
+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);
+  }
+
+  /** 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<String, String> properties = genericTable.getProperties();
+    boolean hasLocationClause = properties.get(TableCatalog.PROP_LOCATION) != 
null;
+    boolean hasPathClause = properties.get(TABLE_PATH_KEY) != null;
+    Map<String, String> tableProperties = Maps.newHashMap();
+    tableProperties.putAll(properties);
+    if (!hasPathClause && hasLocationClause) {
+      // 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);
+    return DataSourceV2Utils.getTableFromProvider(
+        provider, property_map, scala.Option$.MODULE$.<StructType>empty());

Review Comment:
   Good point. That is enough, updated



##########
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() {

Review Comment:
   Oh, that was because isUnityCatalog is a lazy val in Delta catalog, i was 
trying to mimic the same behavior, but i don't think that is necessary, removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to