Skip to content

Commit

Permalink
Move the dependency of API v1 in Catalyst (pingcap#2259)
Browse files Browse the repository at this point in the history
  • Loading branch information
shiyuhang0 authored Mar 11, 2022
1 parent 17dda28 commit e215eb7
Show file tree
Hide file tree
Showing 27 changed files with 1,134 additions and 571 deletions.
Original file line number Diff line number Diff line change
@@ -1 +1 @@
com.pingcap.tispark.TiDBDataSource
com.pingcap.tispark.v2.TiDBTableProvider
4 changes: 3 additions & 1 deletion core/src/main/scala/com/pingcap/tispark/DefaultSource.scala
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,9 @@

package com.pingcap.tispark

import com.pingcap.tispark.v2.TiDBTableProvider

/**
* Spark Data Source API requires that the class name must be `DefaultSource`
*/
class DefaultSource extends TiDBDataSource {}
class DefaultSource extends TiDBTableProvider {}
83 changes: 0 additions & 83 deletions core/src/main/scala/com/pingcap/tispark/TiDBDataSource.scala

This file was deleted.

35 changes: 7 additions & 28 deletions core/src/main/scala/com/pingcap/tispark/utils/ReflectionUtil.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,9 @@
package com.pingcap.tispark.utils

import com.pingcap.tispark.TiSparkInfo
import org.apache.spark.sql.{SparkSession, TiContext}
import org.apache.spark.sql.{SparkSession, Strategy, TiContext}
import org.apache.spark.sql.catalyst.expressions.BasicExpression.TiExpression
import org.apache.spark.sql.catalyst.expressions.{Alias, ExprId, Expression, SortOrder}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
import org.slf4j.LoggerFactory

import java.io.File
Expand Down Expand Up @@ -60,16 +57,8 @@ object ReflectionUtil {
private val SPARK_WRAPPER_CLASS = "com.pingcap.tispark.SparkWrapper"
private val TI_BASIC_EXPRESSION_CLASS =
"org.apache.spark.sql.catalyst.expressions.TiBasicExpression"
private val TI_RESOLUTION_RULE_CLASS =
"org.apache.spark.sql.extensions.TiResolutionRule"
private val TI_RESOLUTION_RULE_V2_CLASS =
"org.apache.spark.sql.extensions.TiResolutionRuleV2"
private val TI_AUTHORIZATION_RULE_CLASS =
"org.apache.spark.sql.extensions.TiAuthorizationRule"
private val TI_PARSER_CLASS =
"org.apache.spark.sql.extensions.TiParser"
private val TI_DDL_RULE_CLASS =
"org.apache.spark.sql.extensions.TiDDLRule"
private val TI_STRATEGY_CLASS =
"org.apache.spark.sql.extensions.TiStrategy"

def newAlias(child: Expression, name: String): Alias = {
classLoader
Expand Down Expand Up @@ -111,23 +100,13 @@ object ReflectionUtil {
.asInstanceOf[Option[TiExpression]]
}

def newTiResolutionRuleV2(
def newTiStrategy(
getOrCreateTiContext: SparkSession => TiContext,
sparkSession: SparkSession): Rule[LogicalPlan] = {
sparkSession: SparkSession): Strategy = {
classLoader
.loadClass(TI_RESOLUTION_RULE_V2_CLASS)
.loadClass(TI_STRATEGY_CLASS)
.getDeclaredConstructor(classOf[SparkSession => TiContext], classOf[SparkSession])
.newInstance(getOrCreateTiContext, sparkSession)
.asInstanceOf[Rule[LogicalPlan]]
}

def newTiAuthRule(
getOrCreateTiContext: SparkSession => TiContext,
sparkSession: SparkSession): Rule[LogicalPlan] = {
classLoader
.loadClass(TI_AUTHORIZATION_RULE_CLASS)
.getDeclaredConstructor(classOf[SparkSession => TiContext], classOf[SparkSession])
.newInstance(getOrCreateTiContext, sparkSession)
.asInstanceOf[Rule[LogicalPlan]]
.asInstanceOf[Strategy]
}
}
179 changes: 179 additions & 0 deletions core/src/main/scala/com/pingcap/tispark/v2/TiDBTable.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,179 @@
/*
* Copyright 2021 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.pingcap.tispark.v2

import com.pingcap.tikv.{TiConfiguration, TiSession}
import com.pingcap.tikv.exception.TiInternalException
import com.pingcap.tikv.key.Handle
import com.pingcap.tikv.meta.{TiDAGRequest, TiTableInfo, TiTimestamp}
import com.pingcap.tispark.utils.TiUtil
import com.pingcap.tispark.v2.TiDBTable.{getDagRequestToRegionTaskExec, getLogicalPlanToRDD}
import com.pingcap.tispark.write.TiDBOptions
import com.pingcap.tispark.TiTableReference
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.connector.catalog.{SupportsRead, TableCapability}
import org.apache.spark.sql.connector.read.ScanBuilder
import org.apache.spark.sql.execution.{ColumnarCoprocessorRDD, SparkPlan}
import org.apache.spark.sql.tispark.{TiHandleRDD, TiRowRDD}
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.sql.{SQLContext, SparkSession, TiExtensions, execution}

import java.util
import java.util.Collections
import scala.collection.mutable.ListBuffer
import collection.JavaConverters._

case class TiDBTable(
session: TiSession,
tableRef: TiTableReference,
table: TiTableInfo,
var ts: TiTimestamp = null,
options: Option[TiDBOptions] = None)(@transient val sqlContext: SQLContext)
extends SupportsRead {

implicit class IdentifierHelper(identifier: TiTableReference) {
def quoted: String = {
Seq(identifier.databaseName, identifier.tableName).map(quote).mkString(".")
}
private def quote(part: String): String = {
if (part.contains(".") || part.contains("`")) {
s"`${part.replace("`", "``")}`"
} else {
part
}
}
}

override lazy val schema: StructType = TiUtil.getSchemaFromTable(table)

override lazy val properties: util.Map[String, String] = {
if (options.isEmpty) {
Collections.emptyMap()
} else {
options.get.parameters.toMap.asJava
}
}

lazy val isTiFlashReplicaAvailable: Boolean = {
// Note:
// - INFORMATION_SCHEMA.TIFLASH_REPLICA is not present in TiKV or PD,
// it is calculated in TiDB and stored in memory.
// - In order to get those helpful information we have to read them from
// either TiKV or PD and keep them in memory as well.
//
// select * from INFORMATION_SCHEMA.TIFLASH_REPLICA where table_id = $id
// TABLE_SCHEMA, TABLE_NAME, TABLE_ID, REPLICA_COUNT, LOCATION_LABELS, AVAILABLE, PROGRESS
table.getTiflashReplicaInfo != null && table.getTiflashReplicaInfo.isAvailable
}

def databaseName: String = tableRef.databaseName
def tableName: String = tableRef.tableName

override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder =
() => () => schema

override def name(): String = tableRef.quoted

override def capabilities(): util.Set[TableCapability] = {
val capabilities = new util.HashSet[TableCapability]
capabilities.add(TableCapability.BATCH_READ)
capabilities
}

override def toString: String = s"TiDBTable($name)"

def dagRequestToRegionTaskExec(dagRequest: TiDAGRequest, output: Seq[Attribute]): SparkPlan = {
getDagRequestToRegionTaskExec(dagRequest, output, session, sqlContext, tableRef)
}

def logicalPlanToRDD(dagRequest: TiDAGRequest, output: Seq[Attribute]): List[TiRowRDD] = {
getLogicalPlanToRDD(dagRequest, output, session, sqlContext, tableRef)
}
}

object TiDBTable {

private def getDagRequestToRegionTaskExec(
dagRequest: TiDAGRequest,
output: Seq[Attribute],
session: TiSession,
sqlContext: SQLContext,
tableRef: TiTableReference): SparkPlan = {
import scala.collection.JavaConverters._
val ids = dagRequest.getPrunedPhysicalIds.asScala
val tiHandleRDDs = new ListBuffer[TiHandleRDD]()
lazy val attributeRef = Seq(
AttributeReference("RegionId", LongType, nullable = false, Metadata.empty)(),
AttributeReference(
"Handles",
ArrayType(ObjectType(classOf[Handle]), containsNull = false),
nullable = false,
Metadata.empty)())
val tiConf = session.getConf
tiConf.setPartitionPerSplit(TiUtil.getPartitionPerSplit(sqlContext))
ids.foreach(id => {
tiHandleRDDs +=
new TiHandleRDD(
dagRequest,
id,
attributeRef,
tiConf,
tableRef,
session,
sqlContext.sparkSession)
})

// TODO: we may optimize by partitioning the result by region.
// https://github.com/pingcap/tispark/issues/1200
val handlePlan = ColumnarCoprocessorRDD(attributeRef, tiHandleRDDs.toList, fetchHandle = true)
execution.ColumnarRegionTaskExec(
handlePlan,
output,
TiUtil.getChunkBatchSize(sqlContext),
dagRequest,
session.getConf,
session.getTimestamp,
session,
sqlContext.sparkSession)
}

private def getLogicalPlanToRDD(
dagRequest: TiDAGRequest,
output: Seq[Attribute],
session: TiSession,
sqlContext: SQLContext,
tableRef: TiTableReference): List[TiRowRDD] = {
import scala.collection.JavaConverters._
val ids = dagRequest.getPrunedPhysicalIds.asScala
val tiRDDs = new ListBuffer[TiRowRDD]
val tiConf = session.getConf
tiConf.setPartitionPerSplit(TiUtil.getPartitionPerSplit(sqlContext))
ids.foreach(id => {
tiRDDs += new TiRowRDD(
dagRequest.copyReqWithPhysicalId(id),
id,
TiUtil.getChunkBatchSize(sqlContext),
tiConf,
output,
tableRef,
session,
sqlContext.sparkSession)
})
tiRDDs.toList
}

}
Loading

0 comments on commit e215eb7

Please sign in to comment.