Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Clean up redundant code #997

Merged
merged 6 commits into from
Aug 13, 2019
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion .ci/build.groovy
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@ def call(ghprbActualCommit, ghprbPullId, ghprbPullTitle, ghprbPullLink, ghprbPul

catchError {
node ('build') {
def ws = pwd()
deleteDir()
container("java") {
stage('Checkout') {
Expand Down
11 changes: 1 addition & 10 deletions .ci/integration_test.groovy
Original file line number Diff line number Diff line change
Expand Up @@ -14,21 +14,18 @@ def call(ghprbActualCommit, ghprbCommentBody, ghprbPullId, ghprbPullTitle, ghprb
if (m1) {
TIDB_BRANCH = "${m1[0][1]}"
}
m1 = null
println "TIDB_BRANCH=${TIDB_BRANCH}"
// parse pd branch
def m2 = ghprbCommentBody =~ /pd\s*=\s*([^\s\\]+)(\s|\\|$)/
if (m2) {
PD_BRANCH = "${m2[0][1]}"
}
m2 = null
println "PD_BRANCH=${PD_BRANCH}"
// parse tikv branch
def m3 = ghprbCommentBody =~ /tikv\s*=\s*([^\s\\]+)(\s|\\|$)/
if (m3) {
TIKV_BRANCH = "${m3[0][1]}"
}
m3 = null
println "TIKV_BRANCH=${TIKV_BRANCH}"
// parse mvn profile
def m4 = ghprbCommentBody =~ /profile\s*=\s*([^\s\\]+)(\s|\\|$)/
Expand All @@ -41,10 +38,6 @@ def call(ghprbActualCommit, ghprbCommentBody, ghprbPullId, ghprbPullTitle, ghprb
return file.split("\n") as List
}

def remove_last_str = { str ->
return str.substring(0, str.length() - 1)
}

def get_mvn_str = { total_chunks ->
def mvnStr = " -DwildcardSuites="
for (int i = 0 ; i < total_chunks.size() - 1; i++) {
Expand All @@ -65,8 +58,7 @@ def call(ghprbActualCommit, ghprbCommentBody, ghprbPullId, ghprbPullTitle, ghprb
println "${NODE_NAME}"
container("golang") {
deleteDir()
def ws = pwd()


// tidb
def tidb_sha1 = sh(returnStdout: true, script: "curl ${FILE_SERVER_URL}/download/refs/pingcap/tidb/${TIDB_BRANCH}/sha1").trim()
sh "curl ${FILE_SERVER_URL}/download/builds/pingcap/tidb/${tidb_sha1}/centos7/tidb-server.tar.gz | tar xz"
Expand Down Expand Up @@ -166,7 +158,6 @@ def call(ghprbActualCommit, ghprbCommentBody, ghprbPullId, ghprbPullTitle, ghprb
node("test_java") {
println "${NODE_NAME}"
container("java") {
def ws = pwd()
deleteDir()
unstash 'binaries'
unstash 'tispark'
Expand Down
10 changes: 4 additions & 6 deletions core/src/main/scala/com/pingcap/tispark/TiDBRelation.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression}
import org.apache.spark.sql.execution._
import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation}
import org.apache.spark.sql.tispark.{TiHandleRDD, TiRDD}
import org.apache.spark.sql.tispark.{TiHandleRDD, TiRDD, TiRowRDD}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}

Expand All @@ -45,13 +45,13 @@ case class TiDBRelation(session: TiSession,

override def sizeInBytes: Long = tableRef.sizeInBytes

def logicalPlanToRDD(dagRequest: TiDAGRequest): List[TiRDD] = {
def logicalPlanToRDD(dagRequest: TiDAGRequest): List[TiRowRDD] = {
import scala.collection.JavaConverters._
val ids = dagRequest.getIds.asScala
var tiRDDs = new ListBuffer[TiRDD]
var tiRDDs = new ListBuffer[TiRowRDD]
ids.foreach(
id => {
tiRDDs += new TiRDD(
tiRDDs += new TiRowRDD(
dagRequest,
id,
session.getConf,
Expand All @@ -65,7 +65,6 @@ case class TiDBRelation(session: TiSession,
}

def dagRequestToRegionTaskExec(dagRequest: TiDAGRequest, output: Seq[Attribute]): SparkPlan = {
val timestamp = dagRequest.getStartTs
import scala.collection.JavaConverters._
val ids = dagRequest.getIds.asScala
var tiHandleRDDs = new ListBuffer[TiHandleRDD]()
Expand All @@ -77,7 +76,6 @@ case class TiDBRelation(session: TiSession,
id,
session.getConf,
tableRef,
timestamp,
session,
sqlContext.sparkSession
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -212,38 +212,36 @@ object StatisticsHelper {
}
}

private[statistics] def buildHistogramsRequest(histTable: TiTableInfo,
targetTblId: Long,
startTs: TiTimestamp): TiDAGRequest =
private def checkColExists(table: TiTableInfo, column: String): Boolean =
table.getColumns.exists { _.matchName(column) }

private def buildRequest(tableInfo: TiTableInfo,
requiredCols: Seq[String],
targetTblId: Long,
startTs: TiTimestamp): TiDAGRequest = {
TiDAGRequest.Builder
.newBuilder()
.setFullTableScan(histTable)
.setFullTableScan(tableInfo)
.addFilter(
ComparisonBinaryExpression
.equal(ColumnRef.create("table_id"), Constant.create(targetTblId))
)
.addRequiredCols(
histRequiredCols.filter(checkColExists(histTable, _))
requiredCols.filter(checkColExists(tableInfo, _))
)
.setStartTs(startTs)
.build(PushDownType.NORMAL)
}

private def checkColExists(table: TiTableInfo, column: String): Boolean =
table.getColumns.exists { _.matchName(column) }
private[statistics] def buildHistogramsRequest(histTable: TiTableInfo,
targetTblId: Long,
startTs: TiTimestamp): TiDAGRequest =
buildRequest(histTable, histRequiredCols, targetTblId, startTs)

private[statistics] def buildMetaRequest(metaTable: TiTableInfo,
targetTblId: Long,
startTs: TiTimestamp): TiDAGRequest =
TiDAGRequest.Builder
.newBuilder()
.setFullTableScan(metaTable)
.addFilter(
ComparisonBinaryExpression
.equal(ColumnRef.create("table_id"), Constant.create(targetTblId))
)
.addRequiredCols(metaRequiredCols.filter(checkColExists(metaTable, _)))
.setStartTs(startTs)
.build(PushDownType.NORMAL)
buildRequest(metaTable, metaRequiredCols, targetTblId, startTs)

private[statistics] def buildBucketRequest(bucketTable: TiTableInfo,
targetTblId: Long,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,29 +36,23 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow,
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.tispark.{TiHandleRDD, TiRDD}
import org.apache.spark.sql.tispark.{TiHandleRDD, TiRDD, TiRowRDD}
import org.apache.spark.sql.types.{ArrayType, DataType, LongType, Metadata}
import org.apache.spark.sql.{Row, SparkSession}
import org.tikv.kvproto.Coprocessor.KeyRange

import scala.collection.JavaConversions._
import scala.collection.mutable

case class CoprocessorRDD(output: Seq[Attribute], tiRDDs: List[TiRDD]) extends LeafExecNode {

override lazy val metrics: Map[String, SQLMetric] = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")
)

override val nodeName: String = "CoprocessorRDD"
trait LeafExecRDD extends LeafExecNode {
override val outputPartitioning: Partitioning = UnknownPartitioning(0)
override val outputOrdering: Seq[SortOrder] = Nil
private[execution] val tiRDDs: List[TiRDD]

private val internalRDDs: List[RDD[InternalRow]] =
private[execution] val internalRDDs: List[RDD[InternalRow]] =
tiRDDs.map(rdd => RDDConversions.rowToRowRdd(rdd, output.map(_.dataType)))
private lazy val project = UnsafeProjection.create(schema)
private[execution] lazy val project = UnsafeProjection.create(schema)

private def internalRowToUnsafeRowWithIndex(
private[execution] def internalRowToUnsafeRowWithIndex(
numOutputRows: SQLMetric
): (Int, Iterator[InternalRow]) => Iterator[UnsafeRow] =
(index, iter) => {
Expand All @@ -69,71 +63,63 @@ case class CoprocessorRDD(output: Seq[Attribute], tiRDDs: List[TiRDD]) extends L
}
}

protected override def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")

internalRDDs
.map(
rdd =>
ReflectionMapPartitionWithIndexInternal(
rdd,
internalRowToUnsafeRowWithIndex(numOutputRows)
).invoke()
)
.reduce(_ union _)

}

def dagRequest: TiDAGRequest = tiRDDs.head.dagRequest

override def verboseString: String =
if (tiRDDs.size > 1) {
val b = new StringBuilder
val b = new mutable.StringBuilder()
b.append(s"TiSpark $nodeName on partition table:\n")
tiRDDs.zipWithIndex.map {
case (_, i) => b.append(s"partition p$i")
}
b.append(s"with dag request: $dagRequest")
b.toString()
} else {
s"TiSpark $nodeName{$dagRequest}" +
s"TiDB $nodeName{$dagRequest}" +
s"${TiUtil.getReqEstCountStr(dagRequest)}"

}

override def simpleString: String = verboseString
}

case class CoprocessorRDD(output: Seq[Attribute], tiRDDs: List[TiRowRDD]) extends LeafExecRDD {

override lazy val metrics: Map[String, SQLMetric] = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")
)

override val nodeName: String = "CoprocessorRDD"

override protected def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")

internalRDDs
.map(
rdd =>
ReflectionMapPartitionWithIndexInternal(
rdd,
internalRowToUnsafeRowWithIndex(numOutputRows)
).invoke()
)
.reduce(_ union _)
}

override def simpleString: String = verboseString
}

/**
* HandleRDDExec is used for scanning handles from TiKV as a LeafExecNode in index plan.
* Providing handle scan via a TiHandleRDD.
*
* @param tiHandleRDDs handle source
* @param tiRDDs handle source
*/
case class HandleRDDExec(tiHandleRDDs: List[TiHandleRDD]) extends LeafExecNode {
case class HandleRDDExec(tiRDDs: List[TiHandleRDD]) extends LeafExecRDD {
override val nodeName: String = "HandleRDD"

override lazy val metrics: Map[String, SQLMetric] = Map(
"numOutputRegions" -> SQLMetrics.createMetric(sparkContext, "number of regions")
)

override val outputPartitioning: Partitioning = UnknownPartitioning(0)

private val internalRDDs: List[RDD[InternalRow]] =
tiHandleRDDs.map(rdd => RDDConversions.rowToRowRdd(rdd, output.map(_.dataType)))
private lazy val project = UnsafeProjection.create(schema)

private def internalRowToUnsafeRowWithIndex(
numOutputRegions: SQLMetric
): (Int, Iterator[InternalRow]) => Iterator[UnsafeRow] =
(index, iter) => {
project.initialize(index)
iter.map { r =>
numOutputRegions += 1
project(r)
}
}

override protected def doExecute(): RDD[InternalRow] = {
val numOutputRegions = longMetric("numOutputRegions")

Expand All @@ -159,24 +145,6 @@ case class HandleRDDExec(tiHandleRDDs: List[TiHandleRDD]) extends LeafExecNode {
)

override def output: Seq[Attribute] = attributeRef

def dagRequest: TiDAGRequest = tiHandleRDDs.head.dagRequest

override def verboseString: String =
if (tiHandleRDDs.size > 1) {
val b = new mutable.StringBuilder()
b.append(s"TiSpark $nodeName on partition table:\n")
tiHandleRDDs.zipWithIndex.map {
case (_, i) => b.append(s"partition p$i")
}
b.append(s"with dag request: $dagRequest")
b.toString()
} else {
s"TiDB $nodeName{$dagRequest}" +
s"${TiUtil.getReqEstCountStr(dagRequest)}"
}

override def simpleString: String = verboseString
}

/**
Expand Down
38 changes: 6 additions & 32 deletions core/src/main/scala/org/apache/spark/sql/tispark/TiHandleRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -39,21 +39,20 @@ import scala.collection.mutable.ListBuffer
* is a list of primitive long which represents the handles lie in that region.
*
*/
class TiHandleRDD(val dagRequest: TiDAGRequest,
val physicalId: Long,
val tiConf: TiConfiguration,
val tableRef: TiTableReference,
val ts: TiTimestamp,
class TiHandleRDD(override val dagRequest: TiDAGRequest,
override val physicalId: Long,
override val tiConf: TiConfiguration,
override val tableRef: TiTableReference,
@transient private val session: TiSession,
@transient private val sparkSession: SparkSession)
extends RDD[Row](sparkSession.sparkContext, Nil) {
extends TiRDD(dagRequest, physicalId, tiConf, tableRef, session, sparkSession) {

override def compute(split: Partition, context: TaskContext): Iterator[Row] =
new Iterator[Row] {
dagRequest.resolve()
private val tiPartition = split.asInstanceOf[TiPartition]
private val session = TiSession.getInstance(tiConf)
private val snapshot = session.createSnapshot(ts)
private val snapshot = session.createSnapshot(dagRequest.getStartTs)
private[this] val tasks = tiPartition.tasks

private val handleIterator = snapshot.indexHandleRead(dagRequest, tasks)
Expand Down Expand Up @@ -96,29 +95,4 @@ class TiHandleRDD(val dagRequest: TiDAGRequest,
Row.apply(regionId, handleList.toArray())
}
}

override protected def getPartitions: Array[Partition] = {
val keyWithRegionTasks = RangeSplitter
.newSplitter(session.getRegionManager)
.splitRangeByRegion(dagRequest.getRangesByPhysicalId(physicalId))

val hostTasksMap = new mutable.HashMap[String, mutable.Set[RegionTask]]
with mutable.MultiMap[String, RegionTask]

var index = 0
val result = new ListBuffer[TiPartition]
for (task <- keyWithRegionTasks) {
hostTasksMap.addBinding(task.getHost, task)
val tasks = hostTasksMap(task.getHost)
result.append(new TiPartition(index, tasks.toSeq, sparkContext.applicationId))
index += 1
hostTasksMap.remove(task.getHost)
}
// add rest
for (tasks <- hostTasksMap.values) {
result.append(new TiPartition(index, tasks.toSeq, sparkContext.applicationId))
index += 1
}
result.toArray
}
}
Loading