【spark系列5】spark 3.0.1集成delta 0.7.0原理解析--delta如何进行DDL DML操作以及Catalog plugin API

前提

本文基于 spark 3.0.1
delta 0.7.0
我们都知道delta.io是一个给数据湖提供可靠性的开源存储层的软件,关于他的用处,可以参考Delta Lake,让你从复杂的Lambda架构中解放出来,上篇文章我们分析了delta是如何自定义自己的sql,这篇文章我们分析一下delta数据是如何基于Catalog plugin API进行DDL DML sql操作的(spark 3.x以前是不支持的)

分析

delta在0.7.0以前是不能够进行save表操作的,只能存储到文件中,也就是说他的元数据是和spark的其他元数据是分开的,delta是独立存在的,也是不能和其他表进行关联操作的,只有到了delta 0.7.0版本以后,才真正意义上和spark进行了集成,这也得益于spark 3.x的Catalog plugin API 特性。
还是先从delta的configurate sparksession入手,如下:

import org.apache.spark.sql.SparkSession

val spark = SparkSession
  .builder()
  .appName("...")
  .master("...")
  .config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
  .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog")
  .getOrCreate()

对于第二个配置 config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog")
spark configuration,我们可以看到对该spark.sql.catalog.spark_catalog的解释是

A catalog implementation that will be used as the v2 interface to Spark's built-in v1 catalog: spark_catalog. This catalog shares its identifier namespace with the spark_catalog and must be consistent with it; for example, if a table can be loaded by the spark_catalog, this catalog must also return the table metadata. To delegate operations to the spark_catalog, implementations can extend 'CatalogExtension'.

也就是说,通过该配置可以实现元数据的统一性,其实这也是spark社区和delta社区进行交互的一种结果

spark 3.x的Catalog plugin API

为了能搞懂delta为什么能够进行DDL和DML操作,就得先知道spark 3.x的Catalog plugin机制SPARK-31121.

首先是interface CatalogPlugin,该接口是catalog plugin的顶级接口,正如注释所说:

 * A marker interface to provide a catalog implementation for Spark.
 * <p>
 * Implementations can provide catalog functions by implementing additional interfaces for tables,
 * views, and functions.
 * <p>
 * Catalog implementations must implement this marker interface to be loaded by
 * {@link Catalogs#load(String, SQLConf)}. The loader will instantiate catalog classes using the
 * required public no-arg constructor. After creating an instance, it will be configured by calling
 * {@link #initialize(String, CaseInsensitiveStringMap)}.
 * <p>
 * Catalog implementations are registered to a name by adding a configuration option to Spark:
 * {@code spark.sql.catalog.catalog-name=com.example.YourCatalogClass}. All configuration properties
 * in the Spark configuration that share the catalog name prefix,
 * {@code spark.sql.catalog.catalog-name.(key)=(value)} will be passed in the case insensitive
 * string map of options in initialization with the prefix removed.
 * {@code name}, is also passed and is the catalog's name; in this case, "catalog-name".

可以通过spark.sql.catalog.catalog-name=com.example.YourCatalogClass集成到spark中
该类的实现还可以集成其他额外的tables views functions的接口,这里就得提到接口TableCatalog,该类提供了与tables相关的方法:

/**
   * List the tables in a namespace from the catalog.
   * <p>
   * If the catalog supports views, this must return identifiers for only tables and not views.
   *
   * @param namespace a multi-part namespace
   * @return an array of Identifiers for tables
   * @throws NoSuchNamespaceException If the namespace does not exist (optional).
   */
  Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException;

  /**
   * Load table metadata by {@link Identifier identifier} from the catalog.
   * <p>
   * If the catalog supports views and contains a view for the identifier and not a table, this
   * must throw {@link NoSuchTableException}.
   *
   * @param ident a table identifier
   * @return the table's metadata
   * @throws NoSuchTableException If the table doesn't exist or is a view
   */
  Table loadTable(Identifier ident) throws NoSuchTableException;

这样就可以基于TableCatalog开发自己的catalog,从而实现multi-catalog support

还得有个接口DelegatingCatalogExtension,这是个实现了CatalogExtension接口的抽象类,而CatalogExtension继承了TableCatalog, SupportsNamespaces。DeltaCatalog实现了DelegatingCatalogExtension ,这部分后续进行分析。
最后还有一个class CatalogManager,这个类是用来管理CatalogPlugins的,且是线程安全的:

/**
 * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow
 * the caller to look up a catalog by name.
 *
 * There are still many commands (e.g. ANALYZE TABLE) that do not support v2 catalog API. They
 * ignore the current catalog and blindly go to the v1 `SessionCatalog`. To avoid tracking current
 * namespace in both `SessionCatalog` and `CatalogManger`, we let `CatalogManager` to set/get
 * current database of `SessionCatalog` when the current catalog is the session catalog.
 */
// TODO: all commands should look up table from the current catalog. The `SessionCatalog` doesn't
//       need to track current database at all.
private[sql]
class CatalogManager(
    conf: SQLConf,
    defaultSessionCatalog: CatalogPlugin,
    val v1SessionCatalog: SessionCatalog) extends Logging {
 

我们看到CatalogManager管理了v2版本的 CatalogPlugin和v1版本的sessionCatalog,这个是因为历史的原因导致必须得兼容v1版本

那CatalogManager在哪里被调用呢。
我们看一下BaseSessionStateBuilder ,可以看到该类中才是正宗使用CatalogManager的地方:

/**
   * Catalog for managing table and database states. If there is a pre-existing catalog, the state
   * of that catalog (temp tables & current database) will be copied into the new catalog.
   *
   * Note: this depends on the `conf`, `functionRegistry` and `sqlParser` fields.
   */
  protected lazy val catalog: SessionCatalog = {
    val catalog = new SessionCatalog(
      () => session.sharedState.externalCatalog,
      () => session.sharedState.globalTempViewManager,
      functionRegistry,
      conf,
      SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf),
      sqlParser,
      resourceLoader)
    parentState.foreach(_.catalog.copyStateTo(catalog))
    catalog
  }

  protected lazy val v2SessionCatalog = new V2SessionCatalog(catalog, conf)

  protected lazy val catalogManager = new CatalogManager(conf, v2SessionCatalog, catalog)

SessionCatalog 是v1版本的,主要是跟底层的元数据存储通信,以及管理临时视图,udf的,这一部分暂时不分析,重点放到v2版本的sessionCatalog,
我们看一下V2SessionCatalog:

/**
 * A [[TableCatalog]] that translates calls to the v1 SessionCatalog.
 */
class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf)
  extends TableCatalog with SupportsNamespaces {
  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper
  import V2SessionCatalog._

  override val defaultNamespace: Array[String] = Array("default")

  override def name: String = CatalogManager.SESSION_CATALOG_NAME

  // This class is instantiated by Spark, so `initialize` method will not be called.
  override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {}

  override def listTables(namespace: Array[String]): Array[Identifier] = {
    namespace match {
      case Array(db) =>
        catalog
          .listTables(db)
          .map(ident => Identifier.of(Array(ident.database.getOrElse("")), ident.table))
          .toArray
      case _ =>
        throw new NoSuchNamespaceException(namespace)
    }
  }

我们分析一下listTables方法可知,v2的sessionCatalog操作 都是委托给了v1版本的sessionCatalog去操作的,其他的方法也是一样,
而且name默认为CatalogManager.SESSION_CATALOG_NAME,也就是spark_catalog,这里后面也会提到,注意一下。
而且,catalogmanager在逻辑计划中的分析器和优化器中也会用到,因为会用到其中的元数据:

protected def analyzer: Analyzer = new Analyzer(catalogManager, conf) {
...
protected def optimizer: Optimizer = {
    new SparkOptimizer(catalogManager, catalog, experimentalMethods) {
      override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
        super.earlyScanPushDownRules ++ customEarlyScanPushDownRules

      override def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] =
        super.extendedOperatorOptimizationRules ++ customOperatorOptimizationRules
    }
  }

而analyzer和optimizer正是spark sql进行解析的核心中的核心,当然还有物理计划的生成。
那这些analyzer和optimizer是在哪里被调用呢?
我们举一个例子,DataSet中的filter方法就调用了:

 */
  def filter(conditionExpr: String): Dataset[T] = {
    filter(Column(sparkSession.sessionState.sqlParser.parseExpression(conditionExpr)))
  }

sessionState.sqlParser就是刚才所说的sqlParser:

protected lazy val sqlParser: ParserInterface = {
    extensions.buildParser(session, new SparkSqlParser(conf))
  }

只有整个逻辑 从sql解析到使用元数据的数据链路,我们就能大致知道怎么一回事了。

delta的DeltaCatalog

我们回过头来看看,delta的DeltaCatalog是怎么和spark 3.x进行结合的 ,上源码DeltaCatalog

class DeltaCatalog(val spark: SparkSession) extends DelegatingCatalogExtension
  with StagingTableCatalog
  with SupportsPathIdentifier {

  def this() = {
    this(SparkSession.active)
  }
  ...

就如之前所说的DeltaCatalog继承了DelegatingCatalogExtension,从名字可以看出这是一个委托类,那到底是怎么委托的呢以及委托给谁呢?

public abstract class DelegatingCatalogExtension implements CatalogExtension {

  private CatalogPlugin delegate;

  public final void setDelegateCatalog(CatalogPlugin delegate) {
    this.delegate = delegate;
  }

该类中有个setDelegateCatalog方法,该方法在CatalogManager中的loadV2SessionCatalog方法中被调用:

private def loadV2SessionCatalog(): CatalogPlugin = {
    Catalogs.load(SESSION_CATALOG_NAME, conf) match {
      case extension: CatalogExtension =>
        extension.setDelegateCatalog(defaultSessionCatalog)
        extension
      case other => other
    }
  }

而该方法被v2SessionCatalog调用:

private[sql] def v2SessionCatalog: CatalogPlugin = {
    conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog =>
      try {
        catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog())
      } catch {
        case NonFatal(_) =>
          logError(
            "Fail to instantiate the custom v2 session catalog: " + customV2SessionCatalog)
          defaultSessionCatalog
      }
    }.getOrElse(defaultSessionCatalog)
  }

这个就是返回默认的v2版本的SessionCatalog实例,分析一下这个方法:

   首先得到配置项SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION,也就是spark.sql.catalog.spark_catalog配置,
   如果spark配置了的话,就调用loadV2SessionCatalog加载该类,,否则就加载默认的v2SessionCatalog,也就是V2SessionCatalog实例

这里我们就发现了:
delta配置的spark.sql.catalog.spark_catalog为"org.apache.spark.sql.delta.catalog.DeltaCatalog",也就是说,spark中的V2SessionCatalog是DeltaCatalog的实例,而DeltaCatalog的委托给了BaseSessionStateBuilder中的V2SessionCatalog实例。

具体看看DeltaCatalog 的createTable方法,其他的方法类似:

override def createTable(
      ident: Identifier,
      schema: StructType,
      partitions: Array[Transform],
      properties: util.Map[String, String]): Table = {
    if (DeltaSourceUtils.isDeltaDataSourceName(getProvider(properties))) {
      createDeltaTable(
        ident, schema, partitions, properties, sourceQuery = None, TableCreationModes.Create)
    } else {
      super.createTable(ident, schema, partitions, properties)
    }
  }
...
private def createDeltaTable(
      ident: Identifier,
      schema: StructType,
      partitions: Array[Transform],
      properties: util.Map[String, String],
      sourceQuery: Option[LogicalPlan],
      operation: TableCreationModes.CreationMode): Table = {
     ...
    val tableDesc = new CatalogTable(
      identifier = TableIdentifier(ident.name(), ident.namespace().lastOption),
      tableType = tableType,
      storage = storage,
      schema = schema,
      provider = Some("delta"),
      partitionColumnNames = partitionColumns,
      bucketSpec = maybeBucketSpec,
      properties = tableProperties.toMap,
      comment = Option(properties.get("comment")))
    // END: copy-paste from the super method finished.

    val withDb = verifyTableAndSolidify(tableDesc, None)
    ParquetSchemaConverter.checkFieldNames(tableDesc.schema.fieldNames)
    CreateDeltaTableCommand(
      withDb,
      getExistingTableIfExists(tableDesc),
      operation.mode,
      sourceQuery,
      operation,
      tableByPath = isByPath).run(spark)

    loadTable(ident)
      }
 override def loadTable(ident: Identifier): Table = {
    try {
      super.loadTable(ident) match {
        case v1: V1Table if DeltaTableUtils.isDeltaTable(v1.catalogTable) =>
          DeltaTableV2(
            spark,
            new Path(v1.catalogTable.location),
            catalogTable = Some(v1.catalogTable),
            tableIdentifier = Some(ident.toString))
        case o => o
      }
    
  }

  • 判断是否是delta数据源,如果是的话,跳到createDeltaTable方法,否则直接调用super.createTable方法,
  • createDeltaTable先会进行delta特有的CreateDeltaTableCommand.run()命令写入delta数据,之后载loadTable
  • loadTable则会调用super的loadTable,而方法会调用V2SessionCatalog的loadTable,而V2SessionCatalog最终会调用v1版本sessionCatalog的getTableMetadata方法,从而组成V1Table(catalogTable)返回,这样就把delta的元数据信息持久化到了v1 SessionCatalog管理的元数据库中
  • 如果不是delta数据源,则调用super.createTable方法,该方法调用V2SessionCatalog的createTable,而最终还是调用v1版本sessionCatalog的createTable方法

我们这里重点分析了delta数据源到元数据的存储,非delta数据源的代码就没有粘贴过来,有兴趣的自己可以编译源码跟踪一下

我们还得提一下spark.sql.defaultCatalog的默认配置为spark_catalog,也就是sql的默认catalog为spark_catalog,对应到delta的话就是DeltaCatalog。

至此,我们就把delta为什么能够进行DDL和DML的原理结合spark的Catalog plugin API分析了一遍.其实搞懂了这些以后,自己也可以按照DeltaCatalog的方式扩展catalog,只不过是catalog的名字不要为spark_catalog,否则会出现异常信息。如果非要为spark_catalog的话,就得继承DelegatingCatalogExtension类,把所有的元数据信息委托给V2SessionCatalog

最后编辑于
©著作权归作者所有,转载或内容合作请联系作者
  • 序言:七十年代末,一起剥皮案震惊了整个滨河市,随后出现的几起案子,更是在滨河造成了极大的恐慌,老刑警刘岩,带你破解...
    沈念sama阅读 220,492评论 6 513
  • 序言:滨河连续发生了三起死亡事件,死亡现场离奇诡异,居然都是意外死亡,警方通过查阅死者的电脑和手机,发现死者居然都...
    沈念sama阅读 94,048评论 3 396
  • 文/潘晓璐 我一进店门,熙熙楼的掌柜王于贵愁眉苦脸地迎上来,“玉大人,你说我怎么就摊上这事。” “怎么了?”我有些...
    开封第一讲书人阅读 166,927评论 0 358
  • 文/不坏的土叔 我叫张陵,是天一观的道长。 经常有香客问我,道长,这世上最难降的妖魔是什么? 我笑而不...
    开封第一讲书人阅读 59,293评论 1 295
  • 正文 为了忘掉前任,我火速办了婚礼,结果婚礼上,老公的妹妹穿的比我还像新娘。我一直安慰自己,他们只是感情好,可当我...
    茶点故事阅读 68,309评论 6 397
  • 文/花漫 我一把揭开白布。 她就那样静静地躺着,像睡着了一般。 火红的嫁衣衬着肌肤如雪。 梳的纹丝不乱的头发上,一...
    开封第一讲书人阅读 52,024评论 1 308
  • 那天,我揣着相机与录音,去河边找鬼。 笑死,一个胖子当着我的面吹牛,可吹牛的内容都是我干的。 我是一名探鬼主播,决...
    沈念sama阅读 40,638评论 3 420
  • 文/苍兰香墨 我猛地睁开眼,长吁一口气:“原来是场噩梦啊……” “哼!你这毒妇竟也来了?” 一声冷哼从身侧响起,我...
    开封第一讲书人阅读 39,546评论 0 276
  • 序言:老挝万荣一对情侣失踪,失踪者是张志新(化名)和其女友刘颖,没想到半个月后,有当地人在树林里发现了一具尸体,经...
    沈念sama阅读 46,073评论 1 319
  • 正文 独居荒郊野岭守林人离奇死亡,尸身上长有42处带血的脓包…… 初始之章·张勋 以下内容为张勋视角 年9月15日...
    茶点故事阅读 38,188评论 3 340
  • 正文 我和宋清朗相恋三年,在试婚纱的时候发现自己被绿了。 大学时的朋友给我发了我未婚夫和他白月光在一起吃饭的照片。...
    茶点故事阅读 40,321评论 1 352
  • 序言:一个原本活蹦乱跳的男人离奇死亡,死状恐怖,灵堂内的尸体忽然破棺而出,到底是诈尸还是另有隐情,我是刑警宁泽,带...
    沈念sama阅读 35,998评论 5 347
  • 正文 年R本政府宣布,位于F岛的核电站,受9级特大地震影响,放射性物质发生泄漏。R本人自食恶果不足惜,却给世界环境...
    茶点故事阅读 41,678评论 3 331
  • 文/蒙蒙 一、第九天 我趴在偏房一处隐蔽的房顶上张望。 院中可真热闹,春花似锦、人声如沸。这庄子的主人今日做“春日...
    开封第一讲书人阅读 32,186评论 0 23
  • 文/苍兰香墨 我抬头看了看天上的太阳。三九已至,却和暖如春,着一层夹袄步出监牢的瞬间,已是汗流浃背。 一阵脚步声响...
    开封第一讲书人阅读 33,303评论 1 272
  • 我被黑心中介骗来泰国打工, 没想到刚下飞机就差点儿被人妖公主榨干…… 1. 我叫王不留,地道东北人。 一个月前我还...
    沈念sama阅读 48,663评论 3 375
  • 正文 我出身青楼,却偏偏与公主长得像,于是被迫代替她去往敌国和亲。 传闻我的和亲对象是个残疾皇子,可洞房花烛夜当晚...
    茶点故事阅读 45,330评论 2 358

推荐阅读更多精彩内容