1
0

[HUDI-2814] Make Z-index more generic Column-Stats Index (#4106)

This commit is contained in:
Alexey Kudinkin
2021-12-10 14:56:09 -08:00
committed by GitHub
parent 72901a33a1
commit 2d864f7524
23 changed files with 892 additions and 1244 deletions

View File

@@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, Bound
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory}
import org.apache.spark.sql.hudi.DataSkippingUtils.createZIndexLookupFilter
import org.apache.spark.sql.hudi.DataSkippingUtils.createColumnStatsIndexFilterExpr
import org.apache.spark.sql.hudi.HoodieSqlUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
@@ -165,8 +165,8 @@ case class HoodieFileIndex(
/**
* Computes pruned list of candidate base-files' names based on provided list of {@link dataFilters}
* conditions, by leveraging custom Z-order index (Z-index) bearing "min", "max", "num_nulls" statistic
* for all clustered columns
* conditions, by leveraging custom Column Statistics index (col-stats-index) bearing "min", "max",
* "num_nulls" statistics for all clustered columns.
*
* NOTE: This method has to return complete set of candidate files, since only provided candidates will
* ultimately be scanned as part of query execution. Hence, this method has to maintain the
@@ -175,8 +175,8 @@ case class HoodieFileIndex(
* @param queryFilters list of original data filters passed down from querying engine
* @return list of pruned (data-skipped) candidate base-files' names
*/
private def lookupCandidateFilesInZIndex(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try {
val indexPath = metaClient.getZindexPath
private def lookupCandidateFilesInColStatsIndex(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try {
val indexPath = metaClient.getColumnStatsIndexPath
val fs = metaClient.getFs
if (!enableDataSkipping() || !fs.exists(new Path(indexPath)) || queryFilters.isEmpty) {
@@ -203,14 +203,14 @@ case class HoodieFileIndex(
Some(spark.read.load(new Path(indexPath, candidateIndexTables.last).toString))
} catch {
case t: Throwable =>
logError("Failed to read Z-index; skipping", t)
logError("Failed to read col-stats index; skipping", t)
None
}
dataFrameOpt.map(df => {
val indexSchema = df.schema
val indexFilter =
queryFilters.map(createZIndexLookupFilter(_, indexSchema))
queryFilters.map(createColumnStatsIndexFilterExpr(_, indexSchema))
.reduce(And)
logInfo(s"Index filter condition: $indexFilter")
@@ -232,13 +232,13 @@ case class HoodieFileIndex(
df.unpersist()
// NOTE: Z-index isn't guaranteed to have complete set of statistics for every
// NOTE: Col-Stats Index isn't guaranteed to have complete set of statistics for every
// base-file: since it's bound to clustering, which could occur asynchronously
// at arbitrary point in time, and is not likely to touching all of the base files.
// at arbitrary point in time, and is not likely to be touching all of the base files.
//
// To close that gap, we manually compute the difference b/w all indexed (Z-index)
// To close that gap, we manually compute the difference b/w all indexed (by col-stats-index)
// files and all outstanding base-files, and make sure that all base files not
// represented w/in Z-index are included in the output of this method
// represented w/in the index are included in the output of this method
val notIndexedFileNames =
lookupFileNamesMissingFromIndex(allIndexedFileNames)
@@ -260,12 +260,12 @@ case class HoodieFileIndex(
*/
override def listFiles(partitionFilters: Seq[Expression],
dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
// Look up candidate files names in the Z-index, if all of the following conditions are true
// Look up candidate files names in the col-stats index, if all of the following conditions are true
// - Data-skipping is enabled
// - Z-index is present
// - Col-Stats Index is present
// - List of predicates (filters) is present
val candidateFilesNamesOpt: Option[Set[String]] =
lookupCandidateFilesInZIndex(dataFilters) match {
lookupCandidateFilesInColStatsIndex(dataFilters) match {
case Success(opt) => opt
case Failure(e) =>
if (e.isInstanceOf[AnalysisException]) {
@@ -280,7 +280,7 @@ case class HoodieFileIndex(
if (queryAsNonePartitionedTable) {
// Read as Non-Partitioned table
// Filter in candidate files based on the Z-index lookup
// Filter in candidate files based on the col-stats index lookup
val candidateFiles =
allFiles.filter(fileStatus =>
// NOTE: This predicate is true when {@code Option} is empty
@@ -305,7 +305,7 @@ case class HoodieFileIndex(
.filter(_ != null)
.map(_.getFileStatus)
// Filter in candidate files based on the Z-index lookup
// Filter in candidate files based on the col-stats index lookup
val candidateFiles =
baseFileStatuses.filter(fs =>
// NOTE: This predicate is true when {@code Option} is empty

View File

@@ -19,7 +19,7 @@ package org.apache.spark.sql.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hudi.index.zorder.ZOrderingIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor}
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
@@ -39,14 +39,14 @@ import scala.collection.JavaConverters._
object DataSkippingUtils extends Logging {
/**
* Translates provided {@link filterExpr} into corresponding filter-expression for Z-index index table
* Translates provided {@link filterExpr} into corresponding filter-expression for column-stats index index table
* to filter out candidate files that would hold records matching the original filter
*
* @param sourceFilterExpr original filter from query
* @param sourceFilterExpr source table's query's filter expression
* @param indexSchema index table schema
* @return filter for Z-index table
* @return filter for column-stats index's table
*/
def createZIndexLookupFilter(sourceFilterExpr: Expression, indexSchema: StructType): Expression = {
def createColumnStatsIndexFilterExpr(sourceFilterExpr: Expression, indexSchema: StructType): Expression = {
// Try to transform original Source Table's filter expression into
// Column-Stats Index filter expression
tryComposeIndexFilterExpr(sourceFilterExpr, indexSchema) match {
@@ -201,14 +201,14 @@ object DataSkippingUtils extends Logging {
)
case or: Or =>
val resLeft = createZIndexLookupFilter(or.left, indexSchema)
val resRight = createZIndexLookupFilter(or.right, indexSchema)
val resLeft = createColumnStatsIndexFilterExpr(or.left, indexSchema)
val resRight = createColumnStatsIndexFilterExpr(or.right, indexSchema)
Option(Or(resLeft, resRight))
case and: And =>
val resLeft = createZIndexLookupFilter(and.left, indexSchema)
val resRight = createZIndexLookupFilter(and.right, indexSchema)
val resLeft = createColumnStatsIndexFilterExpr(and.left, indexSchema)
val resRight = createColumnStatsIndexFilterExpr(and.right, indexSchema)
Option(And(resLeft, resRight))
@@ -219,10 +219,10 @@ object DataSkippingUtils extends Logging {
//
case Not(And(left: Expression, right: Expression)) =>
Option(createZIndexLookupFilter(Or(Not(left), Not(right)), indexSchema))
Option(createColumnStatsIndexFilterExpr(Or(Not(left), Not(right)), indexSchema))
case Not(Or(left: Expression, right: Expression)) =>
Option(createZIndexLookupFilter(And(Not(left), Not(right)), indexSchema))
Option(createColumnStatsIndexFilterExpr(And(Not(left), Not(right)), indexSchema))
case _: Expression => None
}

View File

@@ -17,7 +17,7 @@
package org.apache.hudi
import org.apache.hudi.index.zorder.ZOrderingIndexHelper
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions.{Expression, Not}
@@ -66,7 +66,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase {
)
val indexSchema =
ZOrderingIndexHelper.composeIndexSchema(
ColumnStatsIndexHelper.composeIndexSchema(
sourceTableSchema.fields.toSeq
.filter(f => indexedCols.contains(f.name))
.asJava
@@ -77,7 +77,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase {
def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = {
val resolvedExpr: Expression = resolveFilterExpr(sourceExpr, sourceTableSchema)
val lookupFilter = DataSkippingUtils.createZIndexLookupFilter(resolvedExpr, indexSchema)
val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema)
val spark2 = spark
import spark2.implicits._
@@ -97,7 +97,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase {
@MethodSource(Array("testStringsLookupFilterExpressionsSource"))
def testStringsLookupFilterExpressions(sourceExpr: Expression, input: Seq[IndexRow], output: Seq[String]): Unit = {
val resolvedExpr = resolveFilterExpr(sourceExpr, sourceTableSchema)
val lookupFilter = DataSkippingUtils.createZIndexLookupFilter(resolvedExpr, indexSchema)
val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema)
val spark2 = spark
import spark2.implicits._

View File

@@ -19,27 +19,17 @@
package org.apache.hudi.functional
import org.apache.hadoop.fs.{LocatedFileStatus, Path}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.index.zorder.ZOrderingIndexHelper
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
import org.apache.spark.sql._
import org.apache.spark.sql.functions.typedLit
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Tag, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.apache.spark.sql.{DataFrame, SparkSession}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test}
import java.sql.{Date, Timestamp}
import scala.collection.JavaConversions._
import scala.util.Random
@Tag("functional")
class TestZOrderLayoutOptimization extends HoodieClientTestBase {
class TestColumnStatsIndex extends HoodieClientTestBase {
var spark: SparkSession = _
val sourceTableSchema =
@@ -53,100 +43,25 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
.add("c7", BinaryType)
.add("c8", ByteType)
val commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@BeforeEach
override def setUp() {
initPath()
initSparkContexts()
spark = sqlContext.sparkSession
initTestDataGenerator()
initFileSystem()
spark = sqlContext.sparkSession
}
@AfterEach
override def tearDown() = {
cleanupSparkContexts()
cleanupTestDataGenerator()
cleanupFileSystem()
}
@ParameterizedTest
@ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ"))
def testZOrderingLayoutClustering(tableType: String): Unit = {
val targetRecordsCount = 10000
// Bulk Insert Operation
val records = recordsToStrings(dataGen.generateInserts("001", targetRecordsCount)).toList
val writeDf: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records, 2))
writeDf.write.format("org.apache.hudi")
.options(commonOpts)
.option("hoodie.compact.inline", "false")
.option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType)
// option for clustering
.option("hoodie.parquet.small.file.limit", "0")
.option("hoodie.clustering.inline", "true")
.option("hoodie.clustering.inline.max.commits", "1")
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", "1073741824")
.option("hoodie.clustering.plan.strategy.small.file.limit", "629145600")
.option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString)
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 * 1024 * 1024L))
.option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_ENABLE.key, "true")
.option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat, begin_lon")
.mode(SaveMode.Overwrite)
.save(basePath)
val hudiMetaClient = HoodieTableMetaClient.builder
.setConf(hadoopConf)
.setBasePath(basePath)
.setLoadActiveTimelineOnLoad(true)
.build
val lastCommit = hudiMetaClient.getActiveTimeline.getAllCommitsTimeline.lastInstant().get()
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, lastCommit.getAction)
assertEquals(HoodieInstant.State.COMPLETED, lastCommit.getState)
val readDf =
spark.read
.format("hudi")
.load(basePath)
val readDfSkip =
spark.read
.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true")
.format("hudi")
.load(basePath)
assertEquals(targetRecordsCount, readDf.count())
assertEquals(targetRecordsCount, readDfSkip.count())
readDf.createOrReplaceTempView("hudi_snapshot_raw")
readDfSkip.createOrReplaceTempView("hudi_snapshot_skipping")
def select(tableName: String) =
spark.sql(s"SELECT * FROM $tableName WHERE begin_lat >= 0.49 AND begin_lat < 0.51 AND begin_lon >= 0.49 AND begin_lon < 0.51")
assertRowsMatch(
select("hudi_snapshot_raw"),
select("hudi_snapshot_skipping")
)
cleanupSparkContexts()
}
@Test
@Disabled
def testZIndexTableComposition(): Unit = {
def testColumnStatsTableComposition(): Unit = {
val inputDf =
// NOTE: Schema here is provided for validation that the input date is in the appropriate format
// NOTE: Schema here is provided for validation that the input date is in the appropriate format
spark.read
.schema(sourceTableSchema)
.parquet(
@@ -158,20 +73,20 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
// {@link TimestampType} is not supported, and will throw -- hence skipping "c4"
val newZIndexTableDf =
ZOrderingIndexHelper.buildZIndexTableFor(
ColumnStatsIndexHelper.buildColumnStatsTableFor(
inputDf.sparkSession,
inputDf.inputFiles.toSeq,
zorderedColsSchemaFields
)
val indexSchema =
ZOrderingIndexHelper.composeIndexSchema(
ColumnStatsIndexHelper.composeIndexSchema(
sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq
)
// Collect Z-index stats manually (reading individual Parquet files)
val manualZIndexTableDf =
buildZIndexTableManually(
buildColumnStatsTableManually(
getClass.getClassLoader.getResource("index/zorder/input-table").toString,
zorderedCols,
indexSchema
@@ -197,12 +112,12 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
@Test
@Disabled
def testZIndexTableMerge(): Unit = {
def testColumnStatsTableMerge(): Unit = {
val testZIndexPath = new Path(basePath, "zindex")
val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8")
val indexSchema =
ZOrderingIndexHelper.composeIndexSchema(
ColumnStatsIndexHelper.composeIndexSchema(
sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq
)
@@ -216,7 +131,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
getClass.getClassLoader.getResource("index/zorder/input-table").toString
)
ZOrderingIndexHelper.updateZIndexFor(
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
firstInputDf.sparkSession,
sourceTableSchema,
firstInputDf.inputFiles.toSeq,
@@ -229,13 +144,13 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
// NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able
// to reliably retrieve it
val initialZIndexTable =
spark.read
.parquet(new Path(testZIndexPath, firstCommitInstance).toString)
spark.read
.parquet(new Path(testZIndexPath, firstCommitInstance).toString)
val expectedInitialZIndexTableDf =
spark.read
.schema(indexSchema)
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString)
spark.read
.schema(indexSchema)
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString)
assertEquals(asJson(sort(expectedInitialZIndexTableDf)), asJson(sort(initialZIndexTable)))
@@ -251,7 +166,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
// Update Z-index table
//
ZOrderingIndexHelper.updateZIndexFor(
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
secondInputDf.sparkSession,
sourceTableSchema,
secondInputDf.inputFiles.toSeq,
@@ -264,8 +179,8 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
// NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able
// to reliably retrieve it
val mergedZIndexTable =
spark.read
.parquet(new Path(testZIndexPath, secondCommitInstance).toString)
spark.read
.parquet(new Path(testZIndexPath, secondCommitInstance).toString)
val expectedMergedZIndexTableDf =
spark.read
@@ -277,7 +192,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
@Test
@Disabled
def testZIndexTablesGarbageCollection(): Unit = {
def testColumnStatsTablesGarbageCollection(): Unit = {
val testZIndexPath = new Path(System.getProperty("java.io.tmpdir"), "zindex")
val fs = testZIndexPath.getFileSystem(spark.sparkContext.hadoopConfiguration)
@@ -287,7 +202,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
)
// Try to save statistics
ZOrderingIndexHelper.updateZIndexFor(
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
inputDf.sparkSession,
sourceTableSchema,
inputDf.inputFiles.toSeq,
@@ -298,7 +213,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
)
// Save again
ZOrderingIndexHelper.updateZIndexFor(
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
inputDf.sparkSession,
sourceTableSchema,
inputDf.inputFiles.toSeq,
@@ -309,7 +224,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
)
// Test old index table being cleaned up
ZOrderingIndexHelper.updateZIndexFor(
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
inputDf.sparkSession,
sourceTableSchema,
inputDf.inputFiles.toSeq,
@@ -324,7 +239,7 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
assertEquals(fs.exists(new Path(testZIndexPath, "4")), true)
}
private def buildZIndexTableManually(tablePath: String, zorderedCols: Seq[String], indexSchema: StructType) = {
private def buildColumnStatsTableManually(tablePath: String, zorderedCols: Seq[String], indexSchema: StructType) = {
val files = {
val it = fs.listFiles(new Path(tablePath), true)
var seq = Seq[LocatedFileStatus]()
@@ -339,17 +254,17 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
val df = spark.read.schema(sourceTableSchema).parquet(file.getPath.toString)
val exprs: Seq[String] =
s"'${typedLit(file.getPath.getName)}' AS file" +:
df.columns
.filter(col => zorderedCols.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}_num_nulls"
)
})
df.columns
.filter(col => zorderedCols.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}_num_nulls"
)
})
df.selectExpr(exprs: _*)
.collect()
@@ -366,10 +281,6 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
.map(_.getString(0))
.mkString("\n")
private def assertRowsMatch(one: DataFrame, other: DataFrame) = {
val rows = one.count()
assert(rows == other.count() && one.intersect(other).count() == rows)
}
private def sort(df: DataFrame): DataFrame = {
// Since upon parsing JSON, Spark re-order columns in lexicographical order
@@ -380,19 +291,4 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase {
.sort("file")
}
def createComplexDataFrame(spark: SparkSession): DataFrame = {
val rdd = spark.sparkContext.parallelize(0 to 1000, 1).map { item =>
val c1 = Integer.valueOf(item)
val c2 = s" ${item}sdc"
val c3 = new java.math.BigDecimal(s"${Random.nextInt(1000)}.${item}")
val c4 = new Timestamp(System.currentTimeMillis())
val c5 = java.lang.Short.valueOf(s"${(item + 16) /10}")
val c6 = Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}")
val c7 = Array(item).map(_.toByte)
val c8 = java.lang.Byte.valueOf("9")
RowFactory.create(c1, c2, c3, c4, c5, c6, c7, c8)
}
spark.createDataFrame(rdd, sourceTableSchema)
}
}

View File

@@ -18,18 +18,16 @@
package org.apache.hudi.functional
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.common.util.{BaseFileUtils, ParquetUtils}
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
import org.apache.spark.OrderingIndexHelper
import org.apache.spark.sql._
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag, Test}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.Arguments.arguments
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
@@ -39,9 +37,20 @@ import scala.collection.JavaConversions._
import scala.util.Random
@Tag("functional")
class TestTableLayoutOptimization extends HoodieClientTestBase {
class TestSpaceCurveLayoutOptimization extends HoodieClientTestBase {
var spark: SparkSession = _
val sourceTableSchema =
new StructType()
.add("c1", IntegerType)
.add("c2", StringType)
.add("c3", DecimalType(9,3))
.add("c4", TimestampType)
.add("c5", ShortType)
.add("c6", DateType)
.add("c7", BinaryType)
.add("c8", ByteType)
val commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
@@ -52,7 +61,8 @@ class TestTableLayoutOptimization extends HoodieClientTestBase {
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@BeforeEach override def setUp() {
@BeforeEach
override def setUp() {
initPath()
initSparkContexts()
spark = sqlContext.sparkSession
@@ -60,15 +70,16 @@ class TestTableLayoutOptimization extends HoodieClientTestBase {
initFileSystem()
}
@AfterEach override def tearDown() = {
@AfterEach
override def tearDown() = {
cleanupSparkContexts()
cleanupTestDataGenerator()
cleanupFileSystem()
}
@ParameterizedTest
@MethodSource(Array("testLayOutParameter"))
def testOptimizewithClustering(tableType: String, optimizeMode: String): Unit = {
@MethodSource(Array("testLayoutOptimizationParameters"))
def testLayoutOptimizationFunctional(tableType: String): Unit = {
val targetRecordsCount = 10000
// Bulk Insert Operation
val records = recordsToStrings(dataGen.generateInserts("001", targetRecordsCount)).toList
@@ -88,11 +99,21 @@ class TestTableLayoutOptimization extends HoodieClientTestBase {
.option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString)
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 * 1024 * 1024L))
.option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_ENABLE.key, "true")
.option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_STRATEGY.key(), optimizeMode)
.option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat, begin_lon")
.mode(SaveMode.Overwrite)
.save(basePath)
val hudiMetaClient = HoodieTableMetaClient.builder
.setConf(hadoopConf)
.setBasePath(basePath)
.setLoadActiveTimelineOnLoad(true)
.build
val lastCommit = hudiMetaClient.getActiveTimeline.getAllCommitsTimeline.lastInstant().get()
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, lastCommit.getAction)
assertEquals(HoodieInstant.State.COMPLETED, lastCommit.getState)
val readDf =
spark.read
.format("hudi")
@@ -119,106 +140,12 @@ class TestTableLayoutOptimization extends HoodieClientTestBase {
)
}
def assertRowsMatch(one: DataFrame, other: DataFrame) = {
private def assertRowsMatch(one: DataFrame, other: DataFrame) = {
val rows = one.count()
assert(rows == other.count() && one.intersect(other).count() == rows)
}
@Test
def testCollectMinMaxStatistics(): Unit = {
val testPath = new Path(System.getProperty("java.io.tmpdir"), "minMax")
val statisticPath = new Path(System.getProperty("java.io.tmpdir"), "stat")
val fs = testPath.getFileSystem(spark.sparkContext.hadoopConfiguration)
val complexDataFrame = createComplexDataFrame(spark)
complexDataFrame.repartition(3).write.mode("overwrite").save(testPath.toString)
val df = spark.read.load(testPath.toString)
try {
// test z-order/hilbert sort for all primitive type
// shoud not throw exception.
OrderingIndexHelper.createOptimizedDataFrameByMapValue(df, "c1,c2,c3,c5,c6,c7,c8", 20, "hilbert").show(1)
OrderingIndexHelper.createOptimizedDataFrameByMapValue(df, "c1,c2,c3,c5,c6,c7,c8", 20, "z-order").show(1)
OrderingIndexHelper.createOptimizeDataFrameBySample(df, "c1,c2,c3,c5,c6,c7,c8", 20, "hilbert").show(1)
OrderingIndexHelper.createOptimizeDataFrameBySample(df, "c1,c2,c3,c5,c6,c7,c8", 20, "z-order").show(1)
try {
// do not support TimeStampType, so if we collect statistics for c4, should throw exception
val colDf = OrderingIndexHelper.getMinMaxValue(df, "c1,c2,c3,c5,c6,c7,c8")
colDf.cache()
assertEquals(colDf.count(), 3)
assertEquals(colDf.take(1)(0).length, 22)
colDf.unpersist()
// try to save statistics
OrderingIndexHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "2", Seq("0", "1"))
// save again
OrderingIndexHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "3", Seq("0", "1", "2"))
// test old index table clean
OrderingIndexHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "4", Seq("0", "1", "3"))
assertEquals(!fs.exists(new Path(statisticPath, "2")), true)
assertEquals(fs.exists(new Path(statisticPath, "3")), true)
// test to save different index, new index on ("c1,c6,c7,c8") should be successfully saved.
OrderingIndexHelper.saveStatisticsInfo(df, "c1,c6,c7,c8", statisticPath.toString, "5", Seq("0", "1", "3", "4"))
assertEquals(fs.exists(new Path(statisticPath, "5")), true)
} finally {
if (fs.exists(testPath)) fs.delete(testPath)
if (fs.exists(statisticPath)) fs.delete(statisticPath)
}
}
}
// test collect min-max statistic info for DateType in the case of multithreading.
// parquet will give a wrong statistic result for DateType in the case of multithreading.
@Test
def testMultiThreadParquetFooterReadForDateType(): Unit = {
// create parquet file with DateType
val rdd = spark.sparkContext.parallelize(0 to 100, 1)
.map(item => RowFactory.create(Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}")))
val df = spark.createDataFrame(rdd, new StructType().add("id", DateType))
val testPath = new Path(System.getProperty("java.io.tmpdir"), "testCollectDateType")
val conf = spark.sparkContext.hadoopConfiguration
val cols = new java.util.ArrayList[String]
cols.add("id")
try {
df.repartition(3).write.mode("overwrite").save(testPath.toString)
val inputFiles = spark.read.load(testPath.toString).inputFiles.sortBy(x => x)
val realResult = new Array[(String, String)](3)
inputFiles.zipWithIndex.foreach { case (f, index) =>
val fileUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).asInstanceOf[ParquetUtils]
val res = fileUtils.readRangeFromParquetMetadata(conf, new Path(f), cols).iterator().next()
realResult(index) = (res.getMinValue.toString, res.getMaxValue.toString)
}
// multi thread read with no lock
val resUseLock = new Array[(String, String)](3)
inputFiles.zipWithIndex.par.foreach { case (f, index) =>
val fileUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).asInstanceOf[ParquetUtils]
val res = fileUtils.readRangeFromParquetMetadata(conf, new Path(f), cols).iterator().next()
resUseLock(index) = (res.getMinValue.toString, res.getMaxValue.toString)
}
// check resUseNoLock,
// We can't guarantee that there must be problems in the case of multithreading.
// In order to make ut pass smoothly, we will not check resUseNoLock.
// check resUseLock
// should pass assert
realResult.zip(resUseLock).foreach { case (realValue, testValue) =>
assert(realValue == testValue, s" expect realValue: ${realValue} but find ${testValue}")
}
} finally {
if (fs.exists(testPath)) fs.delete(testPath)
}
}
def createComplexDataFrame(spark: SparkSession): DataFrame = {
val schema = new StructType()
.add("c1", IntegerType)
.add("c2", StringType)
.add("c3", DecimalType(9,3))
.add("c4", TimestampType)
.add("c5", ShortType)
.add("c6", DateType)
.add("c7", BinaryType)
.add("c8", ByteType)
val rdd = spark.sparkContext.parallelize(0 to 1000, 1).map { item =>
val c1 = Integer.valueOf(item)
val c2 = s" ${item}sdc"
@@ -231,12 +158,12 @@ class TestTableLayoutOptimization extends HoodieClientTestBase {
RowFactory.create(c1, c2, c3, c4, c5, c6, c7, c8)
}
spark.createDataFrame(rdd, schema)
spark.createDataFrame(rdd, sourceTableSchema)
}
}
object TestTableLayoutOptimization {
def testLayOutParameter(): java.util.stream.Stream[Arguments] = {
object TestSpaceCurveLayoutOptimization {
def testLayoutOptimizationParameters(): java.util.stream.Stream[Arguments] = {
java.util.stream.Stream.of(
arguments("COPY_ON_WRITE", "hilbert"),
arguments("COPY_ON_WRITE", "z-order"),

View File

@@ -19,27 +19,38 @@
package org.apache.spark.sql.execution.benchmark
import org.apache.hadoop.fs.Path
import org.apache.spark.OrderingIndexHelper
import org.apache.hudi.config.HoodieClusteringConfig.LayoutOptimizationStrategy
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper
import org.apache.hudi.sort.SpaceCurveSortingHelper
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.hudi.TestHoodieSqlBase
import org.apache.spark.sql.types.{IntegerType, StructField}
import org.junit.jupiter.api.{Disabled, Tag, Test}
import scala.util.Random
import scala.collection.JavaConversions._
@Tag("functional")
object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
def getSkippingPercent(tableName: String, co1: String, co2: String, value1: Int, value2: Int): Unit= {
val minMax = OrderingIndexHelper
.getMinMaxValue(spark.sql(s"select * from ${tableName}"), s"${co1}, ${co2}")
.collect().map(f => (f.getInt(1), f.getInt(2), f.getInt(4), f.getInt(5)))
var c = 0
for (elem <- minMax) {
if ((elem._1 <= value1 && elem._2 >= value1) || (elem._3 <= value2 && elem._4 >= value2)) {
c = c + 1
def evalSkippingPercent(tableName: String, co1: String, co2: String, value1: Int, value2: Int): Unit= {
val sourceTableDF = spark.sql(s"select * from ${tableName}")
val orderedColsTypes = Seq(StructField(co1, IntegerType), StructField(co2, IntegerType))
val colStatsIndexTable = ColumnStatsIndexHelper
.buildColumnStatsTableFor(spark, sourceTableDF.inputFiles.toSeq, orderedColsTypes)
.collect()
.map(f => (f.getInt(1), f.getInt(2), f.getInt(4), f.getInt(5)))
var hits = 0
for (fileStatRow <- colStatsIndexTable) {
if ((fileStatRow._1 <= value1 && fileStatRow._2 >= value1) || (fileStatRow._3 <= value2 && fileStatRow._4 >= value2)) {
hits = hits + 1
}
}
val p = c / minMax.size.toDouble
println(s"for table ${tableName} with query filter: ${co1} = ${value1} or ${co2} = ${value2} we can achieve skipping percent ${1.0 - p}")
val p = hits / colStatsIndexTable.size.toDouble
println(s"For table ${tableName} with query filter: ${co1} = ${value1} or ${co2} = ${value2} we can achieve skipping percent ${1.0 - p} (w/ total files ${colStatsIndexTable.size})")
}
/*
@@ -48,6 +59,8 @@ object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
for table table_hilbert_sort_byMap with query filter: c1_int = 500000 or c2_int = 500000 we can achieve skipping percent 0.855
for table table_hilbert_sort_bySample with query filter: c1_int = 500000 or c2_int = 500000 we can achieve skipping percent 0.83
*/
@Test
@Disabled
def runNormalTableSkippingBenchMark(): Unit = {
withTempDir { f =>
withTempTable("table_z_sort_byMap", "table_z_sort_bySample", "table_hilbert_sort_byMap", "table_hilbert_sort_bySample") {
@@ -55,10 +68,10 @@ object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
// choose median value as filter condition.
// the median value of c1_int is 500000
// the median value of c2_int is 500000
getSkippingPercent("table_z_sort_byMap", "c1_int", "c2_int", 500000, 500000)
getSkippingPercent("table_z_sort_bySample", "c1_int", "c2_int", 500000, 500000)
getSkippingPercent("table_hilbert_sort_byMap", "c1_int", "c2_int", 500000, 500000)
getSkippingPercent("table_hilbert_sort_bySample", "c1_int", "c2_int", 500000, 500000)
evalSkippingPercent("table_z_sort_byMap", "c1_int", "c2_int", 500000, 500000)
evalSkippingPercent("table_z_sort_bySample", "c1_int", "c2_int", 500000, 500000)
evalSkippingPercent("table_hilbert_sort_byMap", "c1_int", "c2_int", 500000, 500000)
evalSkippingPercent("table_hilbert_sort_bySample", "c1_int", "c2_int", 500000, 500000)
}
}
}
@@ -69,6 +82,8 @@ object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
for table table_hilbert_sort_byMap_skew with query filter: c1_int = 5000 or c2_int = 500000 we can achieve skipping percent 0.05500000000000005
for table table_hilbert_sort_bySample_skew with query filter: c1_int = 5000 or c2_int = 500000 we can achieve skipping percent 0.84
*/
@Test
@Disabled
def runSkewTableSkippingBenchMark(): Unit = {
withTempDir { f =>
withTempTable("table_z_sort_byMap_skew", "table_z_sort_bySample_skew", "table_hilbert_sort_byMap_skew", "table_hilbert_sort_bySample_skew") {
@@ -77,19 +92,14 @@ object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
// choose median value as filter condition.
// the median value of c1_int is 5000
// the median value of c2_int is 500000
getSkippingPercent("table_z_sort_byMap_skew", "c1_int", "c2_int", 5000, 500000)
getSkippingPercent("table_z_sort_bySample_skew", "c1_int", "c2_int", 5000, 500000)
getSkippingPercent("table_hilbert_sort_byMap_skew", "c1_int", "c2_int", 5000, 500000)
getSkippingPercent("table_hilbert_sort_bySample_skew", "c1_int", "c2_int", 5000, 500000)
evalSkippingPercent("table_z_sort_byMap_skew", "c1_int", "c2_int", 5000, 500000)
evalSkippingPercent("table_z_sort_bySample_skew", "c1_int", "c2_int", 5000, 500000)
evalSkippingPercent("table_hilbert_sort_byMap_skew", "c1_int", "c2_int", 5000, 500000)
evalSkippingPercent("table_hilbert_sort_bySample_skew", "c1_int", "c2_int", 5000, 500000)
}
}
}
def main(args: Array[String]): Unit = {
runNormalTableSkippingBenchMark()
runSkewTableSkippingBenchMark()
}
def withTempTable(tableNames: String*)(f: => Unit): Unit = {
try f finally tableNames.foreach(spark.catalog.dropTempView)
}
@@ -97,11 +107,11 @@ object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
def prepareInterTypeTable(tablePath: Path, numRows: Int, col1Range: Int = 1000000, col2Range: Int = 1000000, skewed: Boolean = false): Unit = {
import spark.implicits._
val df = spark.range(numRows).map(_ => (Random.nextInt(col1Range), Random.nextInt(col2Range))).toDF("c1_int", "c2_int")
val dfOptimizeByMap = OrderingIndexHelper.createOptimizedDataFrameByMapValue(df, "c1_int, c2_int", 200, "z-order")
val dfOptimizeBySample = OrderingIndexHelper.createOptimizeDataFrameBySample(df, "c1_int, c2_int", 200, "z-order")
val dfOptimizeByMap = SpaceCurveSortingHelper.orderDataFrameByMappingValues(df, LayoutOptimizationStrategy.ZORDER, Seq("c1_int", "c2_int"), 200)
val dfOptimizeBySample = SpaceCurveSortingHelper.orderDataFrameBySamplingValues(df, LayoutOptimizationStrategy.ZORDER, Seq("c1_int", "c2_int"), 200)
val dfHilbertOptimizeByMap = OrderingIndexHelper.createOptimizedDataFrameByMapValue(df, "c1_int, c2_int", 200, "hilbert")
val dfHilbertOptimizeBySample = OrderingIndexHelper.createOptimizeDataFrameBySample(df, "c1_int, c2_int", 200, "hilbert")
val dfHilbertOptimizeByMap = SpaceCurveSortingHelper.orderDataFrameByMappingValues(df, LayoutOptimizationStrategy.HILBERT, Seq("c1_int", "c2_int"), 200)
val dfHilbertOptimizeBySample = SpaceCurveSortingHelper.orderDataFrameBySamplingValues(df, LayoutOptimizationStrategy.HILBERT, Seq("c1_int", "c2_int"), 200)
saveAsTable(dfOptimizeByMap, tablePath, if (skewed) "z_sort_byMap_skew" else "z_sort_byMap")
saveAsTable(dfOptimizeBySample, tablePath, if (skewed) "z_sort_bySample_skew" else "z_sort_bySample")
@@ -110,7 +120,6 @@ object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
}
def saveAsTable(df: DataFrame, savePath: Path, suffix: String): Unit = {
df.write.mode("overwrite").save(new Path(savePath, suffix).toString)
spark.read.parquet(new Path(savePath, suffix).toString).createOrReplaceTempView("table_" + suffix)
}