1
0

[HUDI-4250][HUDI-4202] Optimize performance of Column Stats Index reading in Data Skipping (#5746)

We provide an alternative way of fetching Column Stats Index within the reading process to avoid the penalty of a more heavy-weight execution scheduled through a Spark engine.
This commit is contained in:
Alexey Kudinkin
2022-07-25 15:36:12 -07:00
committed by GitHub
parent 6e7ac45735
commit e7c8df7e8b
20 changed files with 730 additions and 387 deletions

View File

@@ -369,8 +369,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
metaClient = HoodieTableMetaClient.reload(metaClient)
case class TestCase(enableMetadata: Boolean,
enableColumnStats: Boolean,
enableDataSkipping: Boolean)
enableColumnStats: Boolean,
enableDataSkipping: Boolean,
columnStatsProcessingModeOverride: String = null)
val testCases: Seq[TestCase] =
TestCase(enableMetadata = false, enableColumnStats = false, enableDataSkipping = false) ::
@@ -378,6 +379,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
TestCase(enableMetadata = true, enableColumnStats = false, enableDataSkipping = true) ::
TestCase(enableMetadata = false, enableColumnStats = true, enableDataSkipping = true) ::
TestCase(enableMetadata = true, enableColumnStats = true, enableDataSkipping = true) ::
TestCase(enableMetadata = true, enableColumnStats = true, enableDataSkipping = true, columnStatsProcessingModeOverride = HoodieMetadataConfig.COLUMN_STATS_INDEX_PROCESSING_MODE_IN_MEMORY) ::
TestCase(enableMetadata = true, enableColumnStats = true, enableDataSkipping = true, columnStatsProcessingModeOverride = HoodieMetadataConfig.COLUMN_STATS_INDEX_PROCESSING_MODE_ENGINE) ::
Nil
for (testCase <- testCases) {
@@ -391,7 +394,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
val props = Map[String, String](
"path" -> basePath,
QUERY_TYPE.key -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> testCase.enableDataSkipping.toString
DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> testCase.enableDataSkipping.toString,
HoodieMetadataConfig.COLUMN_STATS_INDEX_PROCESSING_MODE_OVERRIDE.key -> testCase.columnStatsProcessingModeOverride
) ++ readMetadataOpts
val fileIndex = HoodieFileIndex(spark, metaClient, Option.empty, props, NoopCache)

View File

@@ -31,12 +31,12 @@ import org.apache.hudi.functional.TestColumnStatsIndex.ColumnStatsTestCase
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions}
import org.apache.spark.sql._
import org.apache.spark.sql.functions.typedLit
import org.apache.spark.sql.functions.{col, lit, 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.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.{Arguments, ArgumentsSource, MethodSource, ValueSource}
import java.math.BigInteger
import java.sql.{Date, Timestamp}
@@ -44,7 +44,7 @@ import scala.collection.JavaConverters._
import scala.util.Random
@Tag("functional")
class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSupport {
class TestColumnStatsIndex extends HoodieClientTestBase {
var spark: SparkSession = _
val sourceTableSchema =
@@ -119,35 +119,31 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
.fromProperties(toProperties(metadataOpts))
.build()
val requestedColumns: Seq[String] = {
// Providing empty seq of columns to [[readColumnStatsIndex]] will lead to the whole
// MT to be read, and subsequently filtered
if (testCase.readFullMetadataTable) Seq.empty
else sourceTableSchema.fieldNames
}
val requestedColumns: Seq[String] = sourceTableSchema.fieldNames
val colStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, requestedColumns)
val transposedColStatsDF = transposeColumnStatsIndex(spark, colStatsDF, sourceTableSchema.fieldNames, sourceTableSchema)
val columnStatsIndex = new ColumnStatsIndexSupport(spark, sourceTableSchema, metadataConfig, metaClient)
val expectedColStatsSchema = composeIndexSchema(sourceTableSchema.fieldNames, sourceTableSchema)
// Match against expected column stats table
val expectedColStatsIndexTableDf =
spark.read
.schema(expectedColStatsSchema)
.json(getClass.getClassLoader.getResource("index/colstats/column-stats-index-table.json").toString)
columnStatsIndex.loadTransposed(requestedColumns, testCase.shouldReadInMemory) { transposedColStatsDF =>
// Match against expected column stats table
val expectedColStatsIndexTableDf =
spark.read
.schema(expectedColStatsSchema)
.json(getClass.getClassLoader.getResource("index/colstats/column-stats-index-table.json").toString)
assertEquals(expectedColStatsIndexTableDf.schema, transposedColStatsDF.schema)
// NOTE: We have to drop the `fileName` column as it contains semi-random components
// that we can't control in this test. Nevertheless, since we manually verify composition of the
// ColStats Index by reading Parquet footers from individual Parquet files, this is not an issue
assertEquals(asJson(sort(expectedColStatsIndexTableDf)), asJson(sort(transposedColStatsDF.drop("fileName"))))
assertEquals(expectedColStatsIndexTableDf.schema, transposedColStatsDF.schema)
// NOTE: We have to drop the `fileName` column as it contains semi-random components
// that we can't control in this test. Nevertheless, since we manually verify composition of the
// ColStats Index by reading Parquet footers from individual Parquet files, this is not an issue
assertEquals(asJson(sort(expectedColStatsIndexTableDf)), asJson(sort(transposedColStatsDF.drop("fileName"))))
// Collect Column Stats manually (reading individual Parquet files)
val manualColStatsTableDF =
buildColumnStatsTableManually(basePath, sourceTableSchema.fieldNames, expectedColStatsSchema)
// Collect Column Stats manually (reading individual Parquet files)
val manualColStatsTableDF =
buildColumnStatsTableManually(basePath, sourceTableSchema.fieldNames, sourceTableSchema.fieldNames, expectedColStatsSchema)
assertEquals(asJson(sort(manualColStatsTableDF)), asJson(sort(transposedColStatsDF)))
assertEquals(asJson(sort(manualColStatsTableDF)), asJson(sort(transposedColStatsDF)))
}
// do an upsert and validate
val updateJSONTablePath = getClass.getClassLoader.getResource("index/colstats/another-input-table-json").toString
@@ -166,26 +162,28 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
metaClient = HoodieTableMetaClient.reload(metaClient)
val updatedColStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, requestedColumns)
val transposedUpdatedColStatsDF = transposeColumnStatsIndex(spark, updatedColStatsDF, sourceTableSchema.fieldNames, sourceTableSchema)
val updatedColumnStatsIndex = new ColumnStatsIndexSupport(spark, sourceTableSchema, metadataConfig, metaClient)
val expectedColStatsIndexUpdatedDF =
spark.read
.schema(expectedColStatsSchema)
.json(getClass.getClassLoader.getResource("index/colstats/updated-column-stats-index-table.json").toString)
updatedColumnStatsIndex.loadTransposed(requestedColumns, testCase.shouldReadInMemory) { transposedUpdatedColStatsDF =>
val expectedColStatsIndexUpdatedDF =
spark.read
.schema(expectedColStatsSchema)
.json(getClass.getClassLoader.getResource("index/colstats/updated-column-stats-index-table.json").toString)
assertEquals(expectedColStatsIndexUpdatedDF.schema, transposedUpdatedColStatsDF.schema)
assertEquals(asJson(sort(expectedColStatsIndexUpdatedDF)), asJson(sort(transposedUpdatedColStatsDF.drop("fileName"))))
assertEquals(expectedColStatsIndexUpdatedDF.schema, transposedUpdatedColStatsDF.schema)
assertEquals(asJson(sort(expectedColStatsIndexUpdatedDF)), asJson(sort(transposedUpdatedColStatsDF.drop("fileName"))))
// Collect Column Stats manually (reading individual Parquet files)
val manualUpdatedColStatsTableDF =
buildColumnStatsTableManually(basePath, sourceTableSchema.fieldNames, expectedColStatsSchema)
// Collect Column Stats manually (reading individual Parquet files)
val manualUpdatedColStatsTableDF =
buildColumnStatsTableManually(basePath, sourceTableSchema.fieldNames, sourceTableSchema.fieldNames, expectedColStatsSchema)
assertEquals(asJson(sort(manualUpdatedColStatsTableDF)), asJson(sort(transposedUpdatedColStatsDF)))
assertEquals(asJson(sort(manualUpdatedColStatsTableDF)), asJson(sort(transposedUpdatedColStatsDF)))
}
}
@Test
def testMetadataColumnStatsIndexPartialProjection(): Unit = {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testMetadataColumnStatsIndexPartialProjection(shouldReadInMemory: Boolean): Unit = {
val targetColumnsToIndex = Seq("c1", "c2", "c3")
val metadataOpts = Map(
@@ -235,11 +233,11 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
// These are NOT indexed
val requestedColumns = Seq("c4")
val emptyColStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, requestedColumns)
val emptyTransposedColStatsDF = transposeColumnStatsIndex(spark, emptyColStatsDF, requestedColumns, sourceTableSchema)
val columnStatsIndex = new ColumnStatsIndexSupport(spark, sourceTableSchema, metadataConfig, metaClient)
assertEquals(0, emptyColStatsDF.collect().length)
assertEquals(0, emptyTransposedColStatsDF.collect().length)
columnStatsIndex.loadTransposed(requestedColumns, shouldReadInMemory) { emptyTransposedColStatsDF =>
assertEquals(0, emptyTransposedColStatsDF.collect().length)
}
}
////////////////////////////////////////////////////////////////////////
@@ -252,29 +250,27 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
// We have to include "c1", since we sort the expected outputs by this column
val requestedColumns = Seq("c4", "c1")
val partialColStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, requestedColumns)
val partialTransposedColStatsDF = transposeColumnStatsIndex(spark, partialColStatsDF, requestedColumns, sourceTableSchema)
val targetIndexedColumns = targetColumnsToIndex.intersect(requestedColumns)
val expectedColStatsSchema = composeIndexSchema(targetIndexedColumns, sourceTableSchema)
val expectedColStatsSchema = composeIndexSchema(requestedColumns.sorted, sourceTableSchema)
// Match against expected column stats table
val expectedColStatsIndexTableDf =
spark.read
.schema(expectedColStatsSchema)
.json(getClass.getClassLoader.getResource("index/colstats/partial-column-stats-index-table.json").toString)
assertEquals(expectedColStatsIndexTableDf.schema, partialTransposedColStatsDF.schema)
// NOTE: We have to drop the `fileName` column as it contains semi-random components
// that we can't control in this test. Nevertheless, since we manually verify composition of the
// ColStats Index by reading Parquet footers from individual Parquet files, this is not an issue
assertEquals(asJson(sort(expectedColStatsIndexTableDf)), asJson(sort(partialTransposedColStatsDF.drop("fileName"))))
// Collect Column Stats manually (reading individual Parquet files)
val manualColStatsTableDF =
buildColumnStatsTableManually(basePath, targetIndexedColumns, expectedColStatsSchema)
buildColumnStatsTableManually(basePath, requestedColumns, targetColumnsToIndex, expectedColStatsSchema)
assertEquals(asJson(sort(manualColStatsTableDF)), asJson(sort(partialTransposedColStatsDF)))
val columnStatsIndex = new ColumnStatsIndexSupport(spark, sourceTableSchema, metadataConfig, metaClient)
columnStatsIndex.loadTransposed(requestedColumns, shouldReadInMemory) { partialTransposedColStatsDF =>
assertEquals(expectedColStatsIndexTableDf.schema, partialTransposedColStatsDF.schema)
// NOTE: We have to drop the `fileName` column as it contains semi-random components
// that we can't control in this test. Nevertheless, since we manually verify composition of the
// ColStats Index by reading Parquet footers from individual Parquet files, this is not an issue
assertEquals(asJson(sort(expectedColStatsIndexTableDf)), asJson(sort(partialTransposedColStatsDF.drop("fileName"))))
assertEquals(asJson(sort(manualColStatsTableDF)), asJson(sort(partialTransposedColStatsDF)))
}
}
////////////////////////////////////////////////////////////////////////
@@ -307,27 +303,26 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
val requestedColumns = sourceTableSchema.fieldNames
// Nevertheless, the last update was written with a new schema (that is a subset of the original table schema),
// we should be able to read CSI, which will be properly padded (with nulls) after transposition
val updatedColStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, requestedColumns)
val transposedUpdatedColStatsDF = transposeColumnStatsIndex(spark, updatedColStatsDF, requestedColumns, sourceTableSchema)
val targetIndexedColumns = targetColumnsToIndex.intersect(requestedColumns)
val expectedColStatsSchema = composeIndexSchema(targetIndexedColumns, sourceTableSchema)
val expectedColStatsSchema = composeIndexSchema(requestedColumns.sorted, sourceTableSchema)
val expectedColStatsIndexUpdatedDF =
spark.read
.schema(expectedColStatsSchema)
.json(getClass.getClassLoader.getResource("index/colstats/updated-partial-column-stats-index-table.json").toString)
assertEquals(expectedColStatsIndexUpdatedDF.schema, transposedUpdatedColStatsDF.schema)
assertEquals(asJson(sort(expectedColStatsIndexUpdatedDF)), asJson(sort(transposedUpdatedColStatsDF.drop("fileName"))))
// Collect Column Stats manually (reading individual Parquet files)
val manualUpdatedColStatsTableDF =
buildColumnStatsTableManually(basePath, targetIndexedColumns, expectedColStatsSchema)
buildColumnStatsTableManually(basePath, requestedColumns, targetColumnsToIndex, expectedColStatsSchema)
assertEquals(asJson(sort(manualUpdatedColStatsTableDF)), asJson(sort(transposedUpdatedColStatsDF)))
val columnStatsIndex = new ColumnStatsIndexSupport(spark, sourceTableSchema, metadataConfig, metaClient)
// Nevertheless, the last update was written with a new schema (that is a subset of the original table schema),
// we should be able to read CSI, which will be properly padded (with nulls) after transposition
columnStatsIndex.loadTransposed(requestedColumns, shouldReadInMemory) { transposedUpdatedColStatsDF =>
assertEquals(expectedColStatsIndexUpdatedDF.schema, transposedUpdatedColStatsDF.schema)
assertEquals(asJson(sort(expectedColStatsIndexUpdatedDF)), asJson(sort(transposedUpdatedColStatsDF.drop("fileName"))))
assertEquals(asJson(sort(manualUpdatedColStatsTableDF)), asJson(sort(transposedUpdatedColStatsDF)))
}
}
}
@@ -370,7 +365,10 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
})
}
private def buildColumnStatsTableManually(tablePath: String, indexedCols: Seq[String], indexSchema: StructType) = {
private def buildColumnStatsTableManually(tablePath: String,
includedCols: Seq[String],
indexedCols: Seq[String],
indexSchema: StructType): DataFrame = {
val files = {
val it = fs.listFiles(new Path(tablePath), true)
var seq = Seq[LocatedFileStatus]()
@@ -387,15 +385,23 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
s"'${typedLit(file.getPath.getName)}' AS file" +:
s"sum(1) AS valueCount" +:
df.columns
.filter(col => indexedCols.contains(col))
.filter(col => includedCols.contains(col))
.flatMap(col => {
val minColName = s"${col}_minValue"
val maxColName = s"${col}_maxValue"
Seq(
s"min($col) AS $minColName",
s"max($col) AS $maxColName",
s"sum(cast(isnull($col) AS long)) AS ${col}_nullCount"
)
if (indexedCols.contains(col)) {
Seq(
s"min($col) AS $minColName",
s"max($col) AS $maxColName",
s"sum(cast(isnull($col) AS long)) AS ${col}_nullCount"
)
} else {
Seq(
s"null AS $minColName",
s"null AS $maxColName",
s"null AS ${col}_nullCount"
)
}
})
df.selectExpr(exprs: _*)
@@ -461,11 +467,13 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup
object TestColumnStatsIndex {
case class ColumnStatsTestCase(forceFullLogScan: Boolean, readFullMetadataTable: Boolean)
case class ColumnStatsTestCase(forceFullLogScan: Boolean, shouldReadInMemory: Boolean)
def testMetadataColumnStatsIndexParams: java.util.stream.Stream[Arguments] =
java.util.stream.Stream.of(
Arguments.arguments(ColumnStatsTestCase(forceFullLogScan = false, readFullMetadataTable = false)),
Arguments.arguments(ColumnStatsTestCase(forceFullLogScan = true, readFullMetadataTable = true))
Arguments.arguments(ColumnStatsTestCase(forceFullLogScan = false, shouldReadInMemory = true)),
Arguments.arguments(ColumnStatsTestCase(forceFullLogScan = false, shouldReadInMemory = false)),
Arguments.arguments(ColumnStatsTestCase(forceFullLogScan = true, shouldReadInMemory = false)),
Arguments.arguments(ColumnStatsTestCase(forceFullLogScan = true, shouldReadInMemory = true))
)
}

View File

@@ -30,7 +30,7 @@ import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, DefaultSo
import org.apache.parquet.hadoop.util.counters.BenchmarkCounter
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.{Dataset, HoodieUnsafeRDDUtils, Row, SaveMode}
import org.apache.spark.sql.{Dataset, HoodieUnsafeUtils, Row, SaveMode}
import org.junit.jupiter.api.Assertions.{assertEquals, fail}
import org.junit.jupiter.api.{Disabled, Tag, Test}
@@ -316,7 +316,7 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with
val (rows, bytesRead) = measureBytesRead { () =>
val rdd = relation.buildScan(targetColumns, Array.empty).asInstanceOf[HoodieUnsafeRDD]
HoodieUnsafeRDDUtils.collect(rdd)
HoodieUnsafeUtils.collect(rdd)
}
val targetRecordCount = tableState.targetRecordCount;