-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-4952] Fixing reading from metadata table when there are no inflight commits #6836
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
Changes from 2 commits
34427d0
23d923e
e246d65
d7fbaa4
8f763c8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,7 +25,7 @@ import org.apache.hudi.common.util.StringUtils | |
| import org.apache.hudi.exception.HoodieException | ||
| import org.apache.hudi.keygen.constant.KeyGeneratorOptions | ||
| import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} | ||
| import org.apache.hudi.metadata.HoodieMetadataPayload | ||
| import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadataUtil} | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.{And, Expression, Literal} | ||
|
|
@@ -74,7 +74,7 @@ case class HoodieFileIndex(spark: SparkSession, | |
| spark = spark, | ||
| metaClient = metaClient, | ||
| schemaSpec = schemaSpec, | ||
| configProperties = getConfigProperties(spark, options), | ||
| configProperties = getConfigProperties(spark, options, metaClient), | ||
| queryPaths = HoodieFileIndex.getQueryPaths(options), | ||
| specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key).map(HoodieSqlCommonUtils.formatQueryInstant), | ||
| fileStatusCache = fileStatusCache | ||
|
|
@@ -293,16 +293,20 @@ object HoodieFileIndex extends Logging { | |
| schema.fieldNames.filter { colName => refs.exists(r => resolver.apply(colName, r.name)) } | ||
| } | ||
|
|
||
| def getConfigProperties(spark: SparkSession, options: Map[String, String]) = { | ||
| private def isFilesPartitionAvailable(metaClient: HoodieTableMetaClient): Boolean = { | ||
| metaClient.getTableConfig.getMetadataPartitions.contains(HoodieTableMetadataUtil.PARTITION_NAME_FILES) | ||
| } | ||
|
|
||
| def getConfigProperties(spark: SparkSession, options: Map[String, String], metaClient: HoodieTableMetaClient) = { | ||
| val sqlConf: SQLConf = spark.sessionState.conf | ||
| val properties = new TypedProperties() | ||
|
|
||
| // To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users | ||
| // would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing. | ||
| properties.setProperty(HoodieMetadataConfig.ENABLE.key(), | ||
| sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(), | ||
| HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString)) | ||
| properties.putAll(options.filter(p => p._2 != null).asJava) | ||
| val isMetadataFilesPartitionAvailable = isFilesPartitionAvailable(metaClient) && sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(), | ||
nsivabalan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString).toBoolean | ||
| properties.setProperty(HoodieMetadataConfig.ENABLE.key(), String.valueOf(isMetadataFilesPartitionAvailable)) | ||
| properties.putAll(options.filter(p => p._2 != null && !p._1.equals(HoodieMetadataConfig.ENABLE.key())).asJava) | ||
|
||
| properties | ||
| } | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.