[jvm-packages] fix executor crashing issue when transforming on xgboost4j-spark-gpu (#8025)

* [jvm-packages] fix executor crashing issue when transforming on xgboost4j-spark-gpu

the API XGBoosterSetParam is not thread-safe. Dring the phase of transforming,
XGBoost runs several transforming tasks at a time, and each of them will set
the "gpu_id" and "predictor" parameters, so if several tasks (multi-threads)
all XGBoosterSetParam simultaneously, it may cause the memory to be corrupted
and cause SIGSEGV.

This PR first get the booster from broadcast and set to the correct gpu_id
and predictor, and then all transforming taskes will use the same booster to
do the transforming.
This commit is contained in:
Bobby Wang
2022-06-24 01:18:41 +08:00
committed by GitHub
parent f0c1b842bf
commit a68580e2a7
4 changed files with 50 additions and 30 deletions

View File

@@ -201,9 +201,9 @@ object PreXGBoost extends PreXGBoostProvider {
val (xgbInput, featuresName) = m.vectorize(dataset)
// predict and turn to Row
val predictFunc =
(broadcastBooster: Broadcast[Booster], dm: DMatrix, originalRowItr: Iterator[Row]) => {
(booster: Booster, dm: DMatrix, originalRowItr: Iterator[Row]) => {
val Array(rawPredictionItr, probabilityItr, predLeafItr, predContribItr) =
m.producePredictionItrs(broadcastBooster, dm)
m.producePredictionItrs(booster, dm)
m.produceResultIterator(originalRowItr, rawPredictionItr, probabilityItr,
predLeafItr, predContribItr)
}
@@ -231,9 +231,9 @@ object PreXGBoost extends PreXGBoostProvider {
// predict and turn to Row
val (xgbInput, featuresName) = m.vectorize(dataset)
val predictFunc =
(broadcastBooster: Broadcast[Booster], dm: DMatrix, originalRowItr: Iterator[Row]) => {
(booster: Booster, dm: DMatrix, originalRowItr: Iterator[Row]) => {
val Array(rawPredictionItr, predLeafItr, predContribItr) =
m.producePredictionItrs(broadcastBooster, dm)
m.producePredictionItrs(booster, dm)
m.produceResultIterator(originalRowItr, rawPredictionItr, predLeafItr, predContribItr)
}
@@ -286,7 +286,7 @@ object PreXGBoost extends PreXGBoostProvider {
cacheInfo)
try {
predictFunc(bBooster, dm, batchRow.iterator)
predictFunc(bBooster.value, dm, batchRow.iterator)
} finally {
batchCnt += 1
dm.delete()

View File

@@ -20,7 +20,6 @@ import ml.dmlc.xgboost4j.scala.spark.params._
import ml.dmlc.xgboost4j.scala.{Booster, DMatrix, EvalTrait, ObjectiveTrait, XGBoost => SXGBoost}
import org.apache.hadoop.fs.Path
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.ml.classification._
import org.apache.spark.ml.linalg._
import org.apache.spark.ml.util._
@@ -329,26 +328,26 @@ class XGBoostClassificationModel private[ml](
}
}
private[scala] def producePredictionItrs(broadcastBooster: Broadcast[Booster], dm: DMatrix):
private[scala] def producePredictionItrs(booster: Booster, dm: DMatrix):
Array[Iterator[Row]] = {
val rawPredictionItr = {
broadcastBooster.value.predict(dm, outPutMargin = true, $(treeLimit)).
booster.predict(dm, outPutMargin = true, $(treeLimit)).
map(Row(_)).iterator
}
val probabilityItr = {
broadcastBooster.value.predict(dm, outPutMargin = false, $(treeLimit)).
booster.predict(dm, outPutMargin = false, $(treeLimit)).
map(Row(_)).iterator
}
val predLeafItr = {
if (isDefined(leafPredictionCol)) {
broadcastBooster.value.predictLeaf(dm, $(treeLimit)).map(Row(_)).iterator
booster.predictLeaf(dm, $(treeLimit)).map(Row(_)).iterator
} else {
Iterator()
}
}
val predContribItr = {
if (isDefined(contribPredictionCol)) {
broadcastBooster.value.predictContrib(dm, $(treeLimit)).map(Row(_)).iterator
booster.predictContrib(dm, $(treeLimit)).map(Row(_)).iterator
} else {
Iterator()
}

View File

@@ -30,7 +30,6 @@ import org.apache.spark.ml.param._
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.ml.util.{DefaultXGBoostParamsReader, DefaultXGBoostParamsWriter, XGBoostWriter}
import org.apache.spark.sql.types.StructType
@@ -298,14 +297,14 @@ class XGBoostRegressionModel private[ml] (
}
}
private[scala] def producePredictionItrs(booster: Broadcast[Booster], dm: DMatrix):
private[scala] def producePredictionItrs(booster: Booster, dm: DMatrix):
Array[Iterator[Row]] = {
val originalPredictionItr = {
booster.value.predict(dm, outPutMargin = false, $(treeLimit)).map(Row(_)).iterator
booster.predict(dm, outPutMargin = false, $(treeLimit)).map(Row(_)).iterator
}
val predLeafItr = {
if (isDefined(leafPredictionCol)) {
booster.value.predictLeaf(dm, $(treeLimit)).
booster.predictLeaf(dm, $(treeLimit)).
map(Row(_)).iterator
} else {
Iterator()
@@ -313,7 +312,7 @@ class XGBoostRegressionModel private[ml] (
}
val predContribItr = {
if (isDefined(contribPredictionCol)) {
booster.value.predictContrib(dm, $(treeLimit)).
booster.predictContrib(dm, $(treeLimit)).
map(Row(_)).iterator
} else {
Iterator()