1
0

[HUDI-2788] Fixing issues w/ Z-order Layout Optimization (#4026)

* Simplyfying, tidying up

* Fixed packaging for `TestOptimizeTable`

* Cleaned up `HoodiFileIndex` file filtering seq;
Removed optimization manually reading Parquet table circumventing Spark

* Refactored `DataSkippingUtils`:
  - Fixed checks to validate all statistics cols are present
  - Fixed some predicates being constructed incorrectly
  - Rewrote comments for easier comprehension, added more notes
  - Tidying up

* Tidying up tests

* `lint`

* Fixing compilation

* `TestOptimizeTable` > `TestTableLayoutOptimization`;
Added assertions to test data skipping paths

* Fixed tests to properly hit data-skipping path

* Fixed pruned files candidates lookup seq to conservatively included all non-indexed files

* Added java-doc

* Fixed compilation
This commit is contained in:
Alexey Kudinkin
2021-11-24 10:10:28 -08:00
committed by GitHub
parent 973f78f5ca
commit 60b23b9797
6 changed files with 289 additions and 181 deletions

View File

@@ -18,28 +18,30 @@
package org.apache.hudi.functional
import java.sql.{Date, Timestamp}
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
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.ZCurveOptimizeHelper
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, Test}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Tag, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import java.sql.{Date, Timestamp}
import scala.collection.JavaConversions._
import scala.util.Random
class TestOptimizeTable extends HoodieClientTestBase {
var spark: SparkSession = null
@Tag("functional")
class TestTableLayoutOptimization extends HoodieClientTestBase {
var spark: SparkSession = _
val commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
@@ -67,11 +69,13 @@ class TestOptimizeTable extends HoodieClientTestBase {
@ParameterizedTest
@ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ"))
def testOptimizewithClustering(tableType: String): Unit = {
def testOptimizeWithClustering(tableType: String): Unit = {
val targetRecordsCount = 10000
// Bulk Insert Operation
val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).toList
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
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)
@@ -83,30 +87,41 @@ class TestOptimizeTable extends HoodieClientTestBase {
.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("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)
assertEquals(1000, spark.read.format("hudi").load(basePath).count())
// use unsorted col as filter.
assertEquals(spark.read
.format("hudi").load(basePath).where("end_lat >= 0 and rider != '1' and weight > 0.0").count(),
spark.read.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true")
.format("hudi").load(basePath).where("end_lat >= 0 and rider != '1' and weight > 0.0").count())
// use sorted col as filter.
assertEquals(spark.read.format("hudi").load(basePath)
.where("begin_lat >= 0.49 and begin_lat < 0.51 and begin_lon >= 0.49 and begin_lon < 0.51").count(),
spark.read.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true")
.format("hudi").load(basePath)
.where("begin_lat >= 0.49 and begin_lat < 0.51 and begin_lon >= 0.49 and begin_lon < 0.51").count())
// use sorted cols and unsorted cols as filter
assertEquals(spark.read.format("hudi").load(basePath)
.where("begin_lat >= 0.49 and begin_lat < 0.51 and end_lat > 0.56").count(),
spark.read.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true")
.format("hudi").load(basePath)
.where("begin_lat >= 0.49 and begin_lat < 0.51 and end_lat > 0.56").count())
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")
)
}
def assertRowsMatch(one: DataFrame, other: DataFrame) = {
val rows = one.count()
assert(rows == other.count() && one.intersect(other).count() == rows)
}
@Test