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

[SW-2622] Avoid Multiple Materialization of Spark DataFrame During Conversion to H2OFrame #2652

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import org.apache.spark.{Partition, TaskContext}

import scala.reflect.ClassTag

private[backend] class H2OAwareRDD[U: ClassTag](nodes: Array[NodeDesc], prev: RDD[U])
private[backend] class H2OAwareRDD[U: ClassTag](val nodes: Array[NodeDesc], prev: RDD[U])
extends H2OAwareBaseRDD[U](nodes, prev) {

override def compute(split: Partition, context: TaskContext): Iterator[U] = prev.compute(split, context)
Expand Down
2 changes: 0 additions & 2 deletions core/src/main/scala/ai/h2o/sparkling/backend/H2OChunk.scala
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,6 @@ private[sparkling] object H2OChunk extends RestCommunication {
node: NodeDesc,
conf: H2OConf,
frameName: String,
numRows: Int,
chunkId: Int,
expectedTypes: Array[ExpectedType],
maxVecSizes: Array[Int]): OutputStream = {
Expand All @@ -67,7 +66,6 @@ private[sparkling] object H2OChunk extends RestCommunication {

val parameters = Map(
"frame_name" -> frameName,
"num_rows" -> numRows,
"chunk_id" -> chunkId,
"expected_types" -> expectedTypesString,
"maximum_vector_sizes" -> maxVecSizesString,
Expand Down
5 changes: 3 additions & 2 deletions core/src/main/scala/ai/h2o/sparkling/backend/NodeDesc.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,9 @@ import water.nbhm.NonBlockingHashMap
* is ID of Spark Executor where corresponding instance is located
* @param hostname hostname of the node
* @param port port of the node
* @param allowedCpus The number of cpus processing tasks on H2O node
*/
case class NodeDesc(nodeId: String, hostname: String, port: Int) {
case class NodeDesc(nodeId: String, hostname: String, port: Int, allowedCpus: Int) {
override def productPrefix = ""

def ipPort(): String = s"$hostname:$port"
Expand All @@ -40,7 +41,7 @@ object NodeDesc {

private def fromH2ONode(node: H2ONode): NodeDesc = {
val ipPort = node.getIpPortString.split(":")
NodeDesc(node.index().toString, ipPort(0), Integer.parseInt(ipPort(1)))
NodeDesc(node.index().toString, ipPort(0), Integer.parseInt(ipPort(1)), node._heartbeat._cpus_allowed)
}

private def intern(node: H2ONode): NodeDesc = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,9 @@ trait SharedBackendConf extends SharedBackendConfExtensions {

def restApiTimeout: Int = sparkConf.getInt(PROP_REST_API_TIMEOUT._1, PROP_REST_API_TIMEOUT._2)

def isNumberOfPartitionsOptimizationEnabled(): Boolean =
sparkConf.getBoolean(PROP_OPTIMIZE_NUM_PARTITIONS._1, PROP_OPTIMIZE_NUM_PARTITIONS._2)

/** Setters */
def setInternalClusterMode(): H2OConf = {
if (runsInExternalClusterMode) {
Expand Down Expand Up @@ -334,6 +337,10 @@ trait SharedBackendConf extends SharedBackendConfExtensions {
def setIcedDir(dir: String): H2OConf = set(PROP_ICED_DIR._1, dir)

def setRestApiTimeout(timeout: Int): H2OConf = set(PROP_REST_API_TIMEOUT._1, timeout.toString)

def setNumberOfPartitionsOptimizationEnabled(): H2OConf = set(PROP_OPTIMIZE_NUM_PARTITIONS._1, true)

def setNumberOfPartitionsOptimizationDisabled(): H2OConf = set(PROP_OPTIMIZE_NUM_PARTITIONS._1, false)
}

object SharedBackendConf {
Expand Down Expand Up @@ -698,6 +705,16 @@ object SharedBackendConf {
"setSessionTimeout(Boolean)",
"Timeout in milliseconds for Rest API requests.")

val PROP_OPTIMIZE_NUM_PARTITIONS: BooleanOption = (
"spark.ext.h2o.conversions.optimize.numPartitions",
true,
"""setNumberOfPartitionsOptimizationEnabled()
|setNumberOfPartitionsOptimizationDisabled()
""".stripMargin,
"If enabled, the conversion method asH2OFrame will repartition the input data frame to " +
"the optimal number of partitions for a given H2O cluster."
)

/** Language of the connected client. */
private[sparkling] val PROP_CLIENT_LANGUAGE: (String, String) = ("spark.ext.h2o.client.language", "scala")
}
58 changes: 22 additions & 36 deletions core/src/main/scala/ai/h2o/sparkling/backend/Writer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,20 +33,20 @@ import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
import org.apache.spark.{ExposeUtils, TaskContext, ml, mllib}

private[backend] class Writer(nodeDesc: NodeDesc, metadata: WriterMetadata, numRows: Int, chunkId: Int)
extends Closeable {
private[backend] class Writer(nodeDesc: NodeDesc, metadata: WriterMetadata, chunkId: Int) extends Closeable {

private val outputStream = H2OChunk.putChunk(
nodeDesc,
metadata.conf,
metadata.frameId,
numRows,
chunkId,
metadata.expectedTypes,
metadata.maxVectorSizes)

private val chunkWriter = new ChunkAutoBufferWriter(outputStream)

def setHasNext(data: Boolean): Unit = put(data)

def put(data: Boolean): Unit = chunkWriter.writeBoolean(data)

def put(data: Byte): Unit = chunkWriter.writeByte(data)
Expand Down Expand Up @@ -91,32 +91,32 @@ private[backend] object Writer {

def convert(rdd: H2OAwareRDD[Row], colNames: Array[String], metadata: WriterMetadata): H2OFrame = {
H2OFrame.initializeFrame(metadata.conf, metadata.frameId, colNames)
val partitionSizes = getNonEmptyPartitionSizes(rdd)
val nonEmptyPartitions = getNonEmptyPartitions(partitionSizes)
val numPartitions = rdd.getNumPartitions

val uploadPlan = getUploadPlan(metadata.conf, nonEmptyPartitions.length)
val operation: SparkJob = perDataFramePartition(metadata, uploadPlan, nonEmptyPartitions, partitionSizes)
val rows = SparkSessionUtils.active.sparkContext.runJob(rdd, operation, nonEmptyPartitions)
val res = new Array[Long](nonEmptyPartitions.size)
val uploadPlan = getUploadPlan(metadata.conf, numPartitions)
val operation: SparkJob = perDataFramePartition(metadata, uploadPlan)
val rows = SparkSessionUtils.active.sparkContext.runJob(rdd, operation)
val res = new Array[Long](numPartitions)
rows.foreach { case (chunkIdx, numRows) => res(chunkIdx) = numRows }
val types = SerdeUtils.expectedTypesToVecTypes(metadata.expectedTypes, metadata.maxVectorSizes)
H2OFrame.finalizeFrame(metadata.conf, metadata.frameId, res, types)
H2OFrame(metadata.frameId)
}

private def perDataFramePartition(
metadata: WriterMetadata,
uploadPlan: UploadPlan,
partitions: Seq[Int],
partitionSizes: Map[Int, Int])(context: TaskContext, it: Iterator[Row]): (Int, Long) = {
val chunkIdx = partitions.indexOf(context.partitionId())
val numRows = partitionSizes(context.partitionId())
private def perDataFramePartition(metadata: WriterMetadata, uploadPlan: UploadPlan)(
context: TaskContext,
it: Iterator[Row]): (Int, Long) = {
val chunkIdx = context.partitionId()
var numRows = 0
val domainBuilder = new CategoricalDomainBuilder(metadata.expectedTypes)
val h2oNode = uploadPlan(chunkIdx)
withResource(new Writer(h2oNode, metadata, numRows, chunkIdx)) { writer =>
withResource(new Writer(h2oNode, metadata, chunkIdx)) { writer =>
it.foreach { row =>
writer.setHasNext(true)
sparkRowToH2ORow(row, writer, metadata, domainBuilder)
numRows += 1
}
writer.setHasNext(false)
}
H2OChunk.putChunkCategoricalDomains(h2oNode, metadata.conf, metadata.frameId, chunkIdx, domainBuilder.getDomains())
(chunkIdx, numRows)
Expand Down Expand Up @@ -161,30 +161,16 @@ private[backend] object Writer {
}
}

private def getNonEmptyPartitionSizes[T](rdd: RDD[T]): Map[Int, Int] = {
rdd
.mapPartitionsWithIndex {
case (idx, it) =>
if (it.nonEmpty) {
Iterator.single((idx, it.size))
} else {
Iterator.empty
}
}
.collect()
.toMap
}

private def getNonEmptyPartitions(partitionSizes: Map[Int, Int]): Seq[Int] = {
partitionSizes.keys.toSeq.sorted
}

private def getUploadPlan(conf: H2OConf, numberOfPartitions: Int): UploadPlan = {
val endpoint = getClusterEndpoint(conf)
val parameters = Map("number_of_chunks" -> numberOfPartitions)
val rawPlan = query[UploadPlanV3](endpoint, Paths.UPLOAD_PLAN, conf, parameters)
rawPlan.layout.map { chunkAssignment =>
chunkAssignment.chunk_id -> NodeDesc(chunkAssignment.node_idx.toString, chunkAssignment.ip, chunkAssignment.port)
chunkAssignment.chunk_id -> NodeDesc(
chunkAssignment.node_idx.toString,
chunkAssignment.ip,
chunkAssignment.port,
chunkAssignment.cpus_allowed)
}.toMap
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ object SparkDataFrameConverter extends Logging {
// we are dealing with Dataset[Row]
val flatDataFrame = flattenDataFrame(df)
val schema = flatDataFrame.schema
val rdd = flatDataFrame.rdd // materialized the data frame
val rdd = flatDataFrame.rdd

val elemMaxSizes = collectMaxElementSizes(rdd, schema)
val vecIndices = collectVectorLikeTypes(schema).toArray
Expand All @@ -56,7 +56,16 @@ object SparkDataFrameConverter extends Logging {
val expectedTypes = DataTypeConverter.determineExpectedTypes(schema)

val uniqueFrameId = frameKeyName.getOrElse("frame_rdd_" + rdd.id + scala.util.Random.nextInt())
val metadata = WriterMetadata(hc.getConf, uniqueFrameId, expectedTypes, maxVecSizes, SparkTimeZone.current())
Writer.convert(new H2OAwareRDD(hc.getH2ONodes(), rdd), colNames, metadata)
val conf = hc.getConf
val h2oNodes = hc.getNodes(hc.getClusterInfo(conf))
val metadata = WriterMetadata(conf, uniqueFrameId, expectedTypes, maxVecSizes, SparkTimeZone.current())

val optimizedRdd = if (conf.isNumberOfPartitionsOptimizationEnabled()) {
val optimalSize = h2oNodes.foldLeft(0)((accumulator, node) => accumulator + node.allowedCpus)
rdd.repartition(optimalSize)
} else {
rdd
}
Writer.convert(new H2OAwareRDD(h2oNodes, optimizedRdd), colNames, metadata)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ object H2OClientUtils extends SharedBackendUtils {
.buildArgs()
}

def startH2OClient(hc: H2OContext, conf: H2OConf, nodes: Array[NodeDesc]): NodeDesc = {
def startH2OClient(hc: H2OContext, conf: H2OConf, nodes: Array[NodeDesc]): Unit = {
if (conf.runsInExternalClusterMode) {
setClientIp(conf)
} else {
Expand All @@ -91,7 +91,6 @@ object H2OClientUtils extends SharedBackendUtils {
}
}
}
NodeDesc(SparkEnv.get.executorId, H2O.SELF_ADDRESS.getHostAddress, H2O.API_PORT)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ trait RestApiUtils extends RestCommunication {
val splits = node.ip_port.split(":")
val ip = splits(0)
val port = splits(1).toInt
NodeDesc(idx.toString, ip, port)
NodeDesc(idx.toString, ip, port, node.cpus_allowed)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,11 @@ class H2ORpcEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint
context.reply(H2O.CLOUD.size())
case GetLeaderNodeMsg =>
val reply = if (H2O.SELF.isLeaderNode) {
Some(NodeDesc(SparkEnv.get.executorId, H2O.SELF_ADDRESS.getHostAddress, H2O.API_PORT))
Some(NodeDesc(
SparkEnv.get.executorId,
H2O.SELF_ADDRESS.getHostAddress,
H2O.API_PORT,
H2O.SELF._heartbeat._cpus_allowed))
} else {
None
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ object InternalH2OBackend extends InternalBackendUtils {
val launcherArgs = toH2OArgs(args)
initializeH2OKerberizedHiveSupport(conf)
H2OStarter.start(launcherArgs, true)
NodeDesc(SparkEnv.get.executorId, H2O.SELF_ADDRESS.getHostAddress, H2O.API_PORT)
NodeDesc(SparkEnv.get.executorId, H2O.SELF_ADDRESS.getHostAddress, H2O.API_PORT, H2O.SELF._heartbeat._cpus_allowed)
}

private def registerNewExecutorListener(hc: H2OContext): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -252,6 +252,19 @@ class DataFrameConverterTestSuite extends FunSuite with SharedH2OTestContext {
h2oFrame.delete()
}

test("DataFrame is evaluated only once during conversion to H2OFrame") {
val numberOfEvaluationsAccumulator = spark.sparkContext.longAccumulator("NumberOfEvaluationsAccumulator")
val baseDF = spark.range(1000).repartition(1)
val finalDF = baseDF.mapPartitions { partitions =>
numberOfEvaluationsAccumulator.add(1); partitions
}

val h2oFrame = hc.asH2OFrame(finalDF)
h2oFrame.delete()

numberOfEvaluationsAccumulator.value shouldEqual 1
}

test("DataFrame[FloatField] to H2OFrame[Numeric]") {
val values = Seq(Float.MinValue, Float.MaxValue, -33.33.toFloat, 200.001.toFloat, -5000.34.toFloat)
val df = sc.parallelize(values).map(v => FloatField(v)).toDF
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,6 @@ final class ChunkServlet extends ServletBase {

private case class PUTRequestParameters(
frameName: String,
numRows: Int,
chunkId: Int,
expectedTypes: Array[ExpectedType],
maxVecSizes: Array[Int],
Expand All @@ -160,24 +159,21 @@ final class ChunkServlet extends ServletBase {
private object PUTRequestParameters {
def parse(request: HttpServletRequest): PUTRequestParameters = {
val frameName = getParameterAsString(request, "frame_name")
val numRowsString = getParameterAsString(request, "num_rows")
val numRows = numRowsString.toInt
val chunkIdString = getParameterAsString(request, "chunk_id")
val chunkId = chunkIdString.toInt
val expectedTypesString = getParameterAsString(request, "expected_types")
val expectedTypes = Base64Encoding.decode(expectedTypesString).map(ExpectedTypes(_))
val maximumVectorSizesString = getParameterAsString(request, "maximum_vector_sizes")
val maxVecSizes = Base64Encoding.decodeToIntArray(maximumVectorSizesString)
val compression = getParameterAsString(request, "compression")
PUTRequestParameters(frameName, numRows, chunkId, expectedTypes, maxVecSizes, compression)
PUTRequestParameters(frameName, chunkId, expectedTypes, maxVecSizes, compression)
}
}

/*
* The method handles handles PUT requests for the path /3/Chunk
* It requires 6 GET parameters
* It requires 5 GET parameters
* - frame_name - a unique string identifier of H2O Frame
* - num_rows - a number of rows forming by the body of the request
* - chunk_id - a unique identifier of the chunk within the H2O Frame
* - expected_type - byte array encoded in Base64 encoding. The types corresponds to the `selected_columns` parameter
* - maximum_vector_sizes - maximum vector sizes for each vector column encoded into Base64 encoding.
Expand All @@ -191,12 +187,7 @@ final class ChunkServlet extends ServletBase {
withResource(request.getInputStream) { inputStream =>
withResource(Compression.decompress(parameters.compression, inputStream)) { decompressed =>
withResource(new ChunkAutoBufferReader(decompressed)) { reader =>
reader.readChunk(
parameters.frameName,
parameters.numRows,
parameters.chunkId,
parameters.expectedTypes,
parameters.maxVecSizes)
reader.readChunk(parameters.frameName, parameters.chunkId, parameters.expectedTypes, parameters.maxVecSizes)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,9 @@ public static class ChunkAssigmentV3 extends SchemaV3<Iced, UploadPlanV3.ChunkAs
direction = API.Direction.OUTPUT)
public int port;

@API(help = "Number of CPUs allowed by process", direction = API.Direction.OUTPUT)
public int cpus_allowed;

public ChunkAssigmentV3() {}

public ChunkAssigmentV3(int id, H2ONode node) {
Expand All @@ -59,6 +62,7 @@ public ChunkAssigmentV3(int id, H2ONode node) {
String[] ipPortArray = node.getIpPortString().split(":");
this.ip = ipPortArray[0];
this.port = new Integer(ipPortArray[1]);
this.cpus_allowed = node._heartbeat._cpus_allowed;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,18 +29,12 @@ final class ChunkAutoBufferReader(val inputStream: InputStream) extends Closeabl
private val buffer = new AutoBuffer(inputStream)
private var isLastNAVar: Boolean = false

def readChunk(
frameName: String,
numRows: Int,
chunkId: Int,
expectedTypes: Array[ExpectedType],
maxVecSizes: Array[Int]): Unit = {
def readChunk(frameName: String, chunkId: Int, expectedTypes: Array[ExpectedType], maxVecSizes: Array[Int]): Unit = {
val vecTypes = SerdeUtils.expectedTypesToVecTypes(expectedTypes, maxVecSizes)
val elementSizes = getElementSizes(expectedTypes, maxVecSizes)
val startPositions = getStartPositions(elementSizes)
val chunks = ChunkUtils.createNewChunks(frameName, vecTypes, chunkId)
var rowIdx = 0
while (rowIdx < numRows) {
while (readHasNext()) {
var typeIdx = 0
while (typeIdx < expectedTypes.length) {
expectedTypes(typeIdx) match {
Expand All @@ -60,7 +54,7 @@ final class ChunkAutoBufferReader(val inputStream: InputStream) extends Closeabl
}
typeIdx += 1
}
rowIdx += 1
//rowIdx += 1
}
ChunkUtils.closeNewChunks(chunks)
}
Expand Down Expand Up @@ -142,6 +136,8 @@ final class ChunkAutoBufferReader(val inputStream: InputStream) extends Closeabl
}
}

def readHasNext() = readBoolean()

def readBoolean(): Boolean = {
val data = buffer.getZ
isLastNAVar = isNA(data)
Expand Down