Skip to content

Commit

Permalink
[HUDI-4584] Fixing SQLConf not being propagated to executor (apache…
Browse files Browse the repository at this point in the history
…#6352)

Fixes `HoodieSparkUtils.createRDD` to make sure `SQLConf` is properly propagated to the executor (required by `AvroSerializer`)
  • Loading branch information
Alexey Kudinkin authored Aug 24, 2022
1 parent 1e162bb commit e5584b3
Show file tree
Hide file tree
Showing 2 changed files with 74 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,13 @@ import org.apache.hudi.common.model.HoodieRecord
import org.apache.spark.SPARK_VERSION
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.SQLConfInjectingRDD
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StringType, StructField, StructType}

import scala.collection.JavaConverters._
import scala.reflect.ClassTag

private[hudi] trait SparkVersionsSupport {
def getSparkVersion: String
Expand Down Expand Up @@ -89,8 +93,12 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport {
// serializer is not able to digest it
val readerAvroSchemaStr = readerAvroSchema.toString
val writerAvroSchemaStr = writerAvroSchema.toString

// NOTE: We're accessing toRdd here directly to avoid [[InternalRow]] to [[Row]] conversion
df.queryExecution.toRdd.mapPartitions { rows =>
// Additionally, we have to explicitly wrap around resulting [[RDD]] into the one
// injecting [[SQLConf]], which by default isn't propgated by Spark to the executor(s).
// [[SQLConf]] is required by [[AvroSerializer]]
injectSQLConf(df.queryExecution.toRdd.mapPartitions { rows =>
if (rows.isEmpty) {
Iterator.empty
} else {
Expand All @@ -108,10 +116,13 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport {

rows.map { ir => transform(convert(ir)) }
}
}
}, SQLConf.get)
}

def getCatalystRowSerDe(structType: StructType) : SparkRowSerDe = {
sparkAdapter.createSparkRowSerDe(structType)
}

private def injectSQLConf[T: ClassTag](rdd: RDD[T], conf: SQLConf): RDD[T] =
new SQLConfInjectingRDD(rdd, conf)
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution

import org.apache.spark.{Partition, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.internal.SQLConf

import scala.reflect.ClassTag

/**
* NOTE: This is a generalized version of of Spark's [[SQLExecutionRDD]]
*
* It is just a wrapper over [[sqlRDD]] which sets and makes effective all the configs from the
* captured [[SQLConf]]
*
* @param sqlRDD the `RDD` generated by the SQL plan
* @param conf the `SQLConf` to apply to the execution of the SQL plan
*/
class SQLConfInjectingRDD[T: ClassTag](var sqlRDD: RDD[T], @transient conf: SQLConf) extends RDD[T](sqlRDD) {
private val sqlConfigs = conf.getAllConfs
private lazy val sqlConfExecutorSide = {
val newConf = new SQLConf()
sqlConfigs.foreach { case (k, v) => newConf.setConfString(k, v) }
newConf
}

override val partitioner = firstParent[InternalRow].partitioner

override def getPartitions: Array[Partition] = firstParent[InternalRow].partitions

override def compute(split: Partition, context: TaskContext): Iterator[T] = {
// If we are in the context of a tracked SQL operation, `SQLExecution.EXECUTION_ID_KEY` is set
// and we have nothing to do here. Otherwise, we use the `SQLConf` captured at the creation of
// this RDD.
if (context.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) == null) {
SQLConf.withExistingConf(sqlConfExecutorSide) {
firstParent[T].iterator(split, context)
}
} else {
firstParent[T].iterator(split, context)
}
}
}

0 comments on commit e5584b3

Please sign in to comment.