Skip to content

Commit

Permalink
[spark] use the internal session catalog in spark generic catalog
Browse files Browse the repository at this point in the history
  • Loading branch information
YannByron committed Mar 7, 2024
1 parent ed322ce commit b0ad302
Show file tree
Hide file tree
Showing 4 changed files with 36 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -63,12 +63,11 @@ public class SparkCatalog extends SparkBaseCatalog {

private static final String PRIMARY_KEY_IDENTIFIER = "primary-key";

private String name = null;
protected Catalog catalog = null;

@Override
public void initialize(String name, CaseInsensitiveStringMap options) {
this.name = name;
this.catalogName = name;
CatalogContext catalogContext =
CatalogContext.create(
Options.fromMap(options),
Expand All @@ -87,11 +86,6 @@ public Catalog paimonCatalog() {
return catalog;
}

@Override
public String name() {
return name;
}

@Override
public String[] defaultNamespace() {
return new String[] {Catalog.DEFAULT_DATABASE};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,25 +25,29 @@
import org.apache.paimon.utils.Preconditions;

import org.apache.hadoop.conf.Configuration;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException;
import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException;
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
import org.apache.spark.sql.catalyst.catalog.ExternalCatalog;
import org.apache.spark.sql.catalyst.catalog.InMemoryCatalog;
import org.apache.spark.sql.connector.catalog.CatalogExtension;
import org.apache.spark.sql.connector.catalog.CatalogPlugin;
import org.apache.spark.sql.catalyst.catalog.SessionCatalog;
import org.apache.spark.sql.connector.catalog.CatalogUtils;
import org.apache.spark.sql.connector.catalog.FunctionCatalog;
import org.apache.spark.sql.connector.catalog.Identifier;
import org.apache.spark.sql.connector.catalog.NamespaceChange;
import org.apache.spark.sql.connector.catalog.SupportsNamespaces;
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.catalog.functions.UnboundFunction;
import org.apache.spark.sql.connector.expressions.Transform;
import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog;
import org.apache.spark.sql.internal.SQLConf;
import org.apache.spark.sql.internal.SessionState;
import org.apache.spark.sql.internal.StaticSQLConf;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
Expand All @@ -56,7 +60,6 @@

import static org.apache.paimon.options.CatalogOptions.METASTORE;
import static org.apache.paimon.options.CatalogOptions.WAREHOUSE;
import static org.apache.paimon.utils.Preconditions.checkNotNull;

/* This file is based on source code from the Iceberg Project (http://iceberg.apache.org/), licensed by the Apache
* Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
Expand All @@ -66,19 +69,16 @@
* A Spark catalog that can also load non-Paimon tables.
*
* <p>Most of the content of this class is referenced from Iceberg's SparkSessionCatalog.
*
* @param <T> CatalogPlugin class to avoid casting to TableCatalog and SupportsNamespaces.
*/
public class SparkGenericCatalog<T extends TableCatalog & SupportsNamespaces>
extends SparkBaseCatalog implements CatalogExtension {
public class SparkGenericCatalog extends SparkBaseCatalog implements FunctionCatalog {

private static final Logger LOG = LoggerFactory.getLogger(SparkGenericCatalog.class);

private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};

private String catalogName = null;
private SparkCatalog sparkCatalog = null;
private T sessionCatalog = null;

private V2SessionCatalog sessionCatalog = null;

@Override
public Catalog paimonCatalog() {
Expand Down Expand Up @@ -219,7 +219,9 @@ public void renameTable(Identifier from, Identifier to)

@Override
public final void initialize(String name, CaseInsensitiveStringMap options) {
Configuration hadoopConf = SparkSession.active().sessionState().newHadoopConf();
SessionState sessionState = SparkSession.active().sessionState();
Configuration hadoopConf = sessionState.newHadoopConf();
SparkConf sparkConf = new SparkConf();
if (options.containsKey(METASTORE.key())
&& options.get(METASTORE.key()).equalsIgnoreCase("hive")) {
String uri = options.get(CatalogOptions.URI.key());
Expand All @@ -242,10 +244,18 @@ public final void initialize(String name, CaseInsensitiveStringMap options) {
this.catalogName = name;
this.sparkCatalog = new SparkCatalog();

this.sparkCatalog.initialize(
name,
autoFillConfigurations(
options, SparkSession.active().sessionState().conf(), hadoopConf));
CaseInsensitiveStringMap newOptions =
autoFillConfigurations(options, sessionState.conf(), hadoopConf);
sparkCatalog.initialize(name, newOptions);

if (!options.isEmpty()) {
for (Map.Entry<String, String> entry : options.entrySet()) {
sparkConf.set("spark.hadoop." + entry.getKey(), entry.getValue());
hadoopConf.set(entry.getKey(), entry.getValue());
}
}
ExternalCatalog externalCatalog = CatalogUtils.buildExternalCatalog(sparkConf, hadoopConf);
this.sessionCatalog = new V2SessionCatalog(new SessionCatalog(externalCatalog));
}

private CaseInsensitiveStringMap autoFillConfigurations(
Expand Down Expand Up @@ -280,31 +290,11 @@ private void fillCommonConfigurations(Map<String, String> options, SQLConf sqlCo
}
}

@Override
@SuppressWarnings("unchecked")
public void setDelegateCatalog(CatalogPlugin sparkSessionCatalog) {
if (sparkSessionCatalog instanceof TableCatalog
&& sparkSessionCatalog instanceof SupportsNamespaces) {
this.sessionCatalog = (T) sparkSessionCatalog;
} else {
throw new IllegalArgumentException("Invalid session catalog: " + sparkSessionCatalog);
}
}

@Override
public String name() {
return catalogName;
}

private boolean usePaimon(String provider) {
return provider == null || SparkSource.NAME().equalsIgnoreCase(provider);
}

private T getSessionCatalog() {
checkNotNull(
sessionCatalog,
"Delegated SessionCatalog is missing. "
+ "Please make sure your are replacing Spark's default catalog, named 'spark_catalog'.");
private V2SessionCatalog getSessionCatalog() {
return sessionCatalog;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,14 @@
/** Spark base catalog. */
public abstract class SparkBaseCatalog
implements TableCatalog, SupportsNamespaces, ProcedureCatalog, WithPaimonCatalog {

protected String catalogName;

@Override
public String name() {
return catalogName;
}

@Override
public Procedure loadProcedure(Identifier identifier) throws NoSuchProcedureException {
if (Catalog.SYSTEM_DATABASE_NAME.equals(identifier.namespace()[0])) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ class PaimonHiveTestBase extends PaimonSparkTestBase {
super.sparkConf
.set("spark.sql.warehouse.dir", tempHiveDBDir.getCanonicalPath)
.set("spark.sql.catalogImplementation", "hive")
.set("spark.sql.catalog.spark_catalog", classOf[SparkGenericCatalog[_]].getName)
.set("spark.sql.catalog.spark_catalog", "org.apache.paimon.spark.SparkGenericCatalog")
.set(s"spark.sql.catalog.$paimonHiveCatalogName", classOf[SparkCatalog].getName)
.set(s"spark.sql.catalog.$paimonHiveCatalogName.metastore", "hive")
.set(s"spark.sql.catalog.$paimonHiveCatalogName.warehouse", tempHiveDBDir.getCanonicalPath)
Expand Down

0 comments on commit b0ad302

Please sign in to comment.