You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by yi...@apache.org on 2022/06/11 21:55:29 UTC
[hudi] 01/02: [HUDI-4205] Fix NullPointerException in HFile reader creation (#5841)
This is an automated email from the ASF dual-hosted git repository.
yihua pushed a commit to branch release-0.11.1-rc2
in repository https://gitbox.apache.org/repos/asf/hudi.git
commit b3c77338bfe932061ddc1858e19a3ba8651d0d0d
Author: Y Ethan Guo <et...@gmail.com>
AuthorDate: Sat Jun 11 14:46:43 2022 -0700
[HUDI-4205] Fix NullPointerException in HFile reader creation (#5841)
Replace SerializableConfiguration with SerializableWritable for broadcasting the hadoop configuration before initializing HFile readers
---
.../src/main/scala/org/apache/hudi/HoodieBaseRelation.scala | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
index 47e391a560..43a2d72733 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala
@@ -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
@@ -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))