1
0

[HUDI-3760] Adding capability to fetch Metadata Records by prefix (#5208)

- Adding capability to fetch Metadata Records by key prefix so that Data Skipping could fetch only Column Stats 
- Index records pertaining to the columns being queried by, instead of reading out whole Index.
- Fixed usages of HFileScanner in HFileReader. few code paths uses cached scanner if available. Other code paths uses its own HFileScanner w/ positional read. 

Brief change log
- Rebasing ColumnStatsIndexSupport to rely on HoodieBackedTableMetadata in lieu of reading t/h Spark DS
- Adding methods enabling key-prefix lookups to HoodiFileReader, HoodieHFileReader
- Wiring key-prefix lookup t/h LogRecordScanner impls
- Cleaning up HoodieHFileReader impl

Co-authored-by: sivabalan <n.siva.b@gmail.com>
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-04-06 09:11:08 -07:00
committed by GitHub
parent 7612549bcc
commit 9e87d164b3
46 changed files with 1387 additions and 698 deletions

View File

@@ -372,7 +372,11 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
val props = Map[String, String](
"path" -> basePath,
QUERY_TYPE.key -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true"
DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true",
// NOTE: Metadata Table has to be enabled on the read path as well
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true"
)
val fileIndex = HoodieFileIndex(spark, metaClient, Option.empty, props, NoopCache)

View File

@@ -22,11 +22,11 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, LocatedFileStatus, Path}
import org.apache.hudi.ColumnStatsIndexSupport.composeIndexSchema
import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD}
import org.apache.hudi.HoodieConversionUtils.toProperties
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.common.util.ParquetUtils
import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig}
import org.apache.hudi.metadata.HoodieTableMetadata
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions}
import org.apache.spark.sql._
@@ -34,6 +34,8 @@ import org.apache.spark.sql.functions.typedLit
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertTrue}
import org.junit.jupiter.api._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import java.math.BigInteger
import java.sql.{Date, Timestamp}
@@ -69,8 +71,9 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
cleanupSparkContexts()
}
@Test
def testMetadataColumnStatsIndex(): Unit = {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testMetadataColumnStatsIndex(forceFullLogScan: Boolean): Unit = {
val opts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
@@ -80,6 +83,7 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true",
HoodieMetadataConfig.ENABLE_FULL_SCAN_LOG_FILES.key -> forceFullLogScan.toString,
HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
)
@@ -104,9 +108,11 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
metaClient = HoodieTableMetaClient.reload(metaClient)
val metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath)
val metadataConfig = HoodieMetadataConfig.newBuilder()
.fromProperties(toProperties(opts))
.build()
val colStatsDF = readColumnStatsIndex(spark, metadataTablePath)
val colStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, sourceTableSchema.fieldNames)
val transposedColStatsDF = transposeColumnStatsIndex(spark, colStatsDF, sourceTableSchema.fieldNames, sourceTableSchema)
val expectedColStatsSchema = composeIndexSchema(sourceTableSchema.fieldNames, sourceTableSchema)
@@ -146,7 +152,7 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
metaClient = HoodieTableMetaClient.reload(metaClient)
val updatedColStatsDF = readColumnStatsIndex(spark, metadataTablePath)
val updatedColStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, sourceTableSchema.fieldNames)
val transposedUpdatedColStatsDF = transposeColumnStatsIndex(spark, updatedColStatsDF, sourceTableSchema.fieldNames, sourceTableSchema)
val expectedColStatsIndexUpdatedDF =

View File

@@ -51,17 +51,20 @@ class TestLayoutOptimization extends HoodieClientTestBase {
.add("c7", BinaryType)
.add("c8", ByteType)
val metadataOpts = Map(
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true"
)
val commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true",
DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
) ++ metadataOpts
@BeforeEach
override def setUp() {
@@ -134,6 +137,7 @@ class TestLayoutOptimization extends HoodieClientTestBase {
val readDfSkip =
spark.read
.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true")
.options(metadataOpts)
.format("hudi")
.load(basePath)

View File

@@ -49,19 +49,21 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn
def testReadability(): Unit = {
val dataGen = new HoodieTestDataGenerator()
val opts: Map[String, String] = commonOpts ++ Map(
val metadataOpts: Map[String, String] = Map(
HoodieMetadataConfig.ENABLE.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true",
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true",
HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true"
)
val combinedOpts: Map[String, String] = commonOpts ++ metadataOpts ++
Map(HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1")
// Insert records
val newRecords = dataGen.generateInserts("001", 100)
val newRecordsDF = parseRecords(recordsToStrings(newRecords).asScala)
newRecordsDF.write.format(hudi)
.options(opts)
.options(combinedOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Append)
.save(basePath)
@@ -71,13 +73,13 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn
val updatedRecordsDF = parseRecords(recordsToStrings(updatedRecords).asScala)
updatedRecordsDF.write.format(hudi)
.options(opts)
.options(combinedOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Append)
.save(basePath)
// Files partition of MT
val filesPartitionDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata/files")
val filesPartitionDF = spark.read.options(metadataOpts).format(hudi).load(s"$basePath/.hoodie/metadata/files")
// Smoke test
filesPartitionDF.show()
@@ -95,7 +97,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn
assertEquals(expectedKeys, keys)
// Column Stats Index partition of MT
val colStatsDF = spark.read.format(hudi).load(s"$basePath/.hoodie/metadata/column_stats")
val colStatsDF = spark.read.options(metadataOpts).format(hudi).load(s"$basePath/.hoodie/metadata/column_stats")
// Smoke test
colStatsDF.show()