1
0

[HUDI-3653] Cleaning up bespoke Column Stats Index implementation (#5062)

This commit is contained in:
Alexey Kudinkin
2022-03-30 10:01:43 -07:00
committed by GitHub
parent 04478a45d9
commit 8b796e9686
4 changed files with 120 additions and 453 deletions

View File

@@ -28,13 +28,16 @@ import org.apache.spark.sql.expressions.UserDefinedFunction
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.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Tag, Test}
import java.math.BigInteger
import java.sql.{Date, Timestamp}
import scala.collection.JavaConverters._
import scala.util.Random
// TODO repurpose to test Column Stats in Metadata Table
@Disabled
@Tag("functional")
class TestColumnStatsIndex extends HoodieClientTestBase {
var spark: SparkSession = _
@@ -80,12 +83,12 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
val zorderedColsSchemaFields = inputDf.schema.fields.filter(f => zorderedCols.contains(f.name)).toSeq
// {@link TimestampType} is not supported, and will throw -- hence skipping "c4"
val newZIndexTableDf =
ColumnStatsIndexHelper.buildColumnStatsTableFor(
inputDf.sparkSession,
inputDf.inputFiles.toSeq.asJava,
zorderedColsSchemaFields.asJava
)
val newZIndexTableDf = null
// ColumnStatsIndexHelper.buildColumnStatsTableFor(
// inputDf.sparkSession,
// inputDf.inputFiles.toSeq.asJava,
// zorderedColsSchemaFields.asJava
// )
val indexSchema =
ColumnStatsIndexHelper.composeIndexSchema(
@@ -137,15 +140,15 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
val firstCommitInstance = "0"
val firstInputDf = spark.read.parquet(firstParquetTablePath)
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
firstInputDf.sparkSession,
sourceTableSchema,
firstInputDf.inputFiles.toSeq.asJava,
zorderedCols.asJava,
testZIndexPath.toString,
firstCommitInstance,
Seq().asJava
)
// ColumnStatsIndexHelper.updateColumnStatsIndexFor(
// firstInputDf.sparkSession,
// sourceTableSchema,
// firstInputDf.inputFiles.toSeq.asJava,
// zorderedCols.asJava,
// testZIndexPath.toString,
// firstCommitInstance,
// Seq().asJava
// )
// NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able
// to reliably retrieve it
@@ -173,18 +176,18 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
.parquet(secondParquetTablePath)
//
// Update Z-index table
// Update Column Stats table
//
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
secondInputDf.sparkSession,
sourceTableSchema,
secondInputDf.inputFiles.toSeq.asJava,
zorderedCols.asJava,
testZIndexPath.toString,
secondCommitInstance,
Seq(firstCommitInstance).asJava
)
// ColumnStatsIndexHelper.updateColumnStatsIndexFor(
// secondInputDf.sparkSession,
// sourceTableSchema,
// secondInputDf.inputFiles.toSeq.asJava,
// zorderedCols.asJava,
// testZIndexPath.toString,
// secondCommitInstance,
// Seq(firstCommitInstance).asJava
// )
// NOTE: We don't need to provide schema upon reading from Parquet, since Spark will be able
// to reliably retrieve it
@@ -200,56 +203,6 @@ class TestColumnStatsIndex extends HoodieClientTestBase {
assertEquals(asJson(sort(expectedMergedZIndexTableDf)), asJson(sort(replace(mergedZIndexTable))))
}
@Test
def testColumnStatsTablesGarbageCollection(): Unit = {
val targetParquetTablePath = tempDir.resolve("index/zorder/input-table").toAbsolutePath.toString
val sourceJSONTablePath = getClass.getClassLoader.getResource("index/zorder/input-table-json").toString
bootstrapParquetInputTableFromJSON(sourceJSONTablePath, targetParquetTablePath)
val inputDf = spark.read.parquet(targetParquetTablePath)
val testColumnStatsIndexPath = new Path(tempDir.resolve("zindex").toAbsolutePath.toString)
val fs = testColumnStatsIndexPath.getFileSystem(spark.sparkContext.hadoopConfiguration)
// Try to save statistics
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
inputDf.sparkSession,
sourceTableSchema,
inputDf.inputFiles.toSeq.asJava,
Seq("c1","c2","c3","c5","c6","c7","c8").asJava,
testColumnStatsIndexPath.toString,
"2",
Seq("0", "1").asJava
)
// Save again
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
inputDf.sparkSession,
sourceTableSchema,
inputDf.inputFiles.toSeq.asJava,
Seq("c1","c2","c3","c5","c6","c7","c8").asJava,
testColumnStatsIndexPath.toString,
"3",
Seq("0", "1", "2").asJava
)
// Test old index table being cleaned up
ColumnStatsIndexHelper.updateColumnStatsIndexFor(
inputDf.sparkSession,
sourceTableSchema,
inputDf.inputFiles.toSeq.asJava,
Seq("c1","c2","c3","c5","c6","c7","c8").asJava,
testColumnStatsIndexPath.toString,
"4",
Seq("0", "1", "3").asJava
)
assertEquals(!fs.exists(new Path(testColumnStatsIndexPath, "2")), true)
assertEquals(!fs.exists(new Path(testColumnStatsIndexPath, "3")), true)
assertEquals(fs.exists(new Path(testColumnStatsIndexPath, "4")), true)
}
@Test
def testParquetMetadataRangeExtraction(): Unit = {
val df = generateRandomDataFrame(spark)

View File

@@ -27,20 +27,20 @@ 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._
import scala.util.Random
@Tag("functional")
object SpaceCurveOptimizeBenchMark extends TestHoodieSqlBase {
object SpaceCurveOptimizeBenchmark extends TestHoodieSqlBase {
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)))
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) {