Skip to content

Commit

Permalink
[HUDI-4205] Fix NullPointerException in HFile reader creation
Browse files Browse the repository at this point in the history
  • Loading branch information
yihua committed Jun 11, 2022
1 parent 2b3a855 commit b69406f
Showing 1 changed file with 4 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,10 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.StringUtils
import org.apache.hudi.common.util.ValidationUtils.checkState
import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema}
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema}
import org.apache.hudi.io.storage.HoodieHFileReader
import org.apache.spark.SerializableWritable
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
Expand Down Expand Up @@ -535,11 +536,10 @@ object HoodieBaseRelation extends SparkAdapterSupport {
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val hadoopConfBroadcast =
spark.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
val hadoopConfBroadcast = spark.sparkContext.broadcast(new SerializableWritable(hadoopConf))

partitionedFile => {
val hadoopConf = hadoopConfBroadcast.value.get()
val hadoopConf = hadoopConfBroadcast.value.value
val reader = new HoodieHFileReader[GenericRecord](hadoopConf, new Path(partitionedFile.filePath),
new CacheConfig(hadoopConf))

Expand Down

0 comments on commit b69406f

Please sign in to comment.