[HUDI-2814] Addressing issues w/ Z-order Layout Optimization (#4060)
* `ZCurveOptimizeHelper` > `ZOrderingIndexHelper`; Moved Z-index helper under `hudi.index.zorder` package * Tidying up `ZOrderingIndexHelper` * Fixing compilation * Fixed index new/original table merging sequence to always prefer values from new index; Cleaned up `HoodieSparkUtils` * Added test for `mergeIndexSql` * Abstracted Z-index name composition w/in `ZOrderingIndexHelper`; * Fixed `DataSkippingUtils` to interrupt prunning in case data filter contains non-indexed column reference * Properly handle exceptions origination during pruning in `HoodieFileIndex` * Make sure no errors are logged upon encountering `AnalysisException` * Cleaned up Z-index updating sequence; Tidying up comments, java-docs; * Fixed Z-index to properly handle changes of the list of clustered columns * Tidying up * `lint` * Suppressing `JavaDocStyle` first sentence check * Fixed compilation * Fixing incorrect `DecimalType` conversion * Refactored test `TestTableLayoutOptimization` - Added Z-index table composition test (against fixtures) - Separated out GC test; Tidying up * Fixed tests re-shuffling column order for Z-Index table `DataFrame` to align w/ the one by one loaded from JSON * Scaffolded `DataTypeUtils` to do basic checks of Spark types; Added proper compatibility checking b/w old/new index-tables * Added test for Z-index tables merging * Fixed import being shaded by creating internal `hudi.util` package * Fixed packaging for `TestOptimizeTable` * Revised `updateMetadataIndex` seq to provide Z-index updating process w/ source table schema * Make sure existing Z-index table schema is sync'd to source table's one * Fixed shaded refs * Fixed tests * Fixed type conversion of Parquet provided metadata values into Spark expected schemas * Fixed `composeIndexSchema` utility to propose proper schema * Added more tests for Z-index: - Checking that Z-index table is built correctly - Checking that Z-index tables are merged correctly (during update) * Fixing source table * Fixing tests to read from Parquet w/ proper schema * Refactored `ParquetUtils` utility reading stats from Parquet footers * Fixed incorrect handling of Decimals extracted from Parquet footers * Worked around issues in javac failign to compile stream's collection * Fixed handling of `Date` type * Fixed handling of `DateType` to be parsed as `LocalDate` * Updated fixture; Make sure test loads Z-index fixture using proper schema * Removed superfluous scheme adjusting when reading from Parquet, since Spark is actually able to perfectly restore schema (given Parquet was previously written by Spark as well) * Fixing race-condition in Parquet's `DateStringifier` trying to share `SimpleDataFormat` object which is inherently not thread-safe * Tidying up * Make sure schema is used upon reading to validate input files are in the appropriate format; Tidying up; * Worked around javac (1.8) inability to infer expression type properly * Updated fixtures; Tidying up * Fixing compilation after rebase * Assert clustering have in Z-order layout optimization testing * Tidying up exception messages * XXX * Added test validating Z-index lookup filter correctness * Added more test-cases; Tidying up * Added tests for string expressions * Fixed incorrect Z-index filter lookup translations * Added more test-cases * Added proper handling on complex negations of AND/OR expressions by pushing NOT operator down into inner expressions for appropriate handling * Added `-target:jvm-1.8` for `hudi-spark` module * Adding more tests * Added tests for non-indexed columns * Properly handle non-indexed columns by falling back to a re-write of containing expression as `TrueLiteral` instead * Fixed tests * Removing the parquet test files and disabling corresponding tests Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -30,21 +30,24 @@ import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.{Column, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate}
|
||||
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, HoodieSqlUtils}
|
||||
import org.apache.spark.sql.hudi.DataSkippingUtils.createZIndexLookupFilter
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{AnalysisException, Column, SparkSession}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
||||
/**
|
||||
* A file index which support partition prune for hoodie snapshot and read-optimized query.
|
||||
@@ -169,16 +172,16 @@ case class HoodieFileIndex(
|
||||
* ultimately be scanned as part of query execution. Hence, this method has to maintain the
|
||||
* invariant of conservatively including every base-file's name, that is NOT referenced in its index.
|
||||
*
|
||||
* @param dataFilters list of original data filters passed down from querying engine
|
||||
* @param queryFilters list of original data filters passed down from querying engine
|
||||
* @return list of pruned (data-skipped) candidate base-files' names
|
||||
*/
|
||||
private def lookupCandidateFilesNamesInZIndex(dataFilters: Seq[Expression]): Option[Set[String]] = {
|
||||
private def lookupCandidateFilesInZIndex(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try {
|
||||
val indexPath = metaClient.getZindexPath
|
||||
val fs = metaClient.getFs
|
||||
|
||||
if (!enableDataSkipping() || !fs.exists(new Path(indexPath)) || dataFilters.isEmpty) {
|
||||
if (!enableDataSkipping() || !fs.exists(new Path(indexPath)) || queryFilters.isEmpty) {
|
||||
// scalastyle:off return
|
||||
return Option.empty
|
||||
return Success(Option.empty)
|
||||
// scalastyle:on return
|
||||
}
|
||||
|
||||
@@ -192,7 +195,7 @@ case class HoodieFileIndex(
|
||||
|
||||
if (candidateIndexTables.isEmpty) {
|
||||
// scalastyle:off return
|
||||
return Option.empty
|
||||
return Success(Option.empty)
|
||||
// scalastyle:on return
|
||||
}
|
||||
|
||||
@@ -207,7 +210,7 @@ case class HoodieFileIndex(
|
||||
dataFrameOpt.map(df => {
|
||||
val indexSchema = df.schema
|
||||
val indexFilter =
|
||||
dataFilters.map(DataSkippingUtils.createZIndexLookupFilter(_, indexSchema))
|
||||
queryFilters.map(createZIndexLookupFilter(_, indexSchema))
|
||||
.reduce(And)
|
||||
|
||||
logInfo(s"Index filter condition: $indexFilter")
|
||||
@@ -221,7 +224,7 @@ case class HoodieFileIndex(
|
||||
.toSet
|
||||
|
||||
val prunedCandidateFileNames =
|
||||
df.filter(new Column(indexFilter))
|
||||
df.where(new Column(indexFilter))
|
||||
.select("file")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
@@ -261,11 +264,22 @@ case class HoodieFileIndex(
|
||||
// - Data-skipping is enabled
|
||||
// - Z-index is present
|
||||
// - List of predicates (filters) is present
|
||||
val candidateFilesNamesOpt: Option[Set[String]] = lookupCandidateFilesNamesInZIndex(dataFilters)
|
||||
val candidateFilesNamesOpt: Option[Set[String]] =
|
||||
lookupCandidateFilesInZIndex(dataFilters) match {
|
||||
case Success(opt) => opt
|
||||
case Failure(e) =>
|
||||
if (e.isInstanceOf[AnalysisException]) {
|
||||
logDebug("Failed to relay provided data filters to Z-index lookup", e)
|
||||
} else {
|
||||
logError("Failed to lookup candidate files in Z-index", e)
|
||||
}
|
||||
Option.empty
|
||||
}
|
||||
|
||||
logDebug(s"Overlapping candidate files (from Z-index): ${candidateFilesNamesOpt.getOrElse(Set.empty)}")
|
||||
|
||||
if (queryAsNonePartitionedTable) { // Read as Non-Partitioned table.
|
||||
if (queryAsNonePartitionedTable) {
|
||||
// Read as Non-Partitioned table
|
||||
// Filter in candidate files based on the Z-index lookup
|
||||
val candidateFiles =
|
||||
allFiles.filter(fileStatus =>
|
||||
@@ -273,9 +287,10 @@ case class HoodieFileIndex(
|
||||
candidateFilesNamesOpt.forall(_.contains(fileStatus.getPath.getName))
|
||||
)
|
||||
|
||||
logInfo(s"Total files : ${allFiles.size}," +
|
||||
s" candidate files after data skipping: ${candidateFiles.size} " +
|
||||
s" skipping percent ${if (allFiles.length != 0) (allFiles.size - candidateFiles.size) / allFiles.size.toDouble else 0}")
|
||||
logInfo(s"Total files : ${allFiles.size}; " +
|
||||
s"candidate files after data skipping: ${candidateFiles.size}; " +
|
||||
s"skipping percent ${if (allFiles.nonEmpty) (allFiles.size - candidateFiles.size) / allFiles.size.toDouble else 0}")
|
||||
|
||||
Seq(PartitionDirectory(InternalRow.empty, candidateFiles))
|
||||
} else {
|
||||
// Prune the partition path by the partition filters
|
||||
@@ -284,27 +299,27 @@ case class HoodieFileIndex(
|
||||
var candidateFileSize = 0
|
||||
|
||||
val result = prunedPartitions.map { partition =>
|
||||
val baseFileStatuses = cachedAllInputFileSlices(partition).map(fileSlice => {
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileStatus
|
||||
} else {
|
||||
null
|
||||
}
|
||||
}).filterNot(_ == null)
|
||||
val baseFileStatuses: Seq[FileStatus] =
|
||||
cachedAllInputFileSlices(partition)
|
||||
.map(fs => fs.getBaseFile.orElse(null))
|
||||
.filter(_ != null)
|
||||
.map(_.getFileStatus)
|
||||
|
||||
// Filter in candidate files based on the Z-index lookup
|
||||
val candidateFiles =
|
||||
baseFileStatuses.filter(fileStatus =>
|
||||
baseFileStatuses.filter(fs =>
|
||||
// NOTE: This predicate is true when {@code Option} is empty
|
||||
candidateFilesNamesOpt.forall(_.contains(fileStatus.getPath.getName)))
|
||||
candidateFilesNamesOpt.forall(_.contains(fs.getPath.getName)))
|
||||
|
||||
totalFileSize += baseFileStatuses.size
|
||||
candidateFileSize += candidateFiles.size
|
||||
PartitionDirectory(partition.values, candidateFiles)
|
||||
}
|
||||
logInfo(s"Total files: ${totalFileSize}," +
|
||||
s" Candidate files after data skipping : ${candidateFileSize} " +
|
||||
s"skipping percent ${if (allFiles.length != 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
|
||||
|
||||
logInfo(s"Total base files: ${totalFileSize}; " +
|
||||
s"candidate files after data skipping : ${candidateFileSize}; " +
|
||||
s"skipping percent ${if (allFiles.nonEmpty) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
|
||||
|
||||
result
|
||||
}
|
||||
}
|
||||
|
||||
@@ -54,7 +54,6 @@ import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession}
|
||||
import org.apache.spark.{SPARK_VERSION, SparkContext}
|
||||
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
@@ -289,7 +288,7 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
def generateSchemaWithoutPartitionColumns(partitionParam: String, schema: Schema): Schema = {
|
||||
val fieldsToRemove = new util.ArrayList[String]()
|
||||
val fieldsToRemove = new java.util.ArrayList[String]()
|
||||
partitionParam.split(",").map(partitionField => partitionField.trim)
|
||||
.filter(s => !s.isEmpty).map(field => fieldsToRemove.add(field))
|
||||
HoodieAvroUtils.removeFields(schema, fieldsToRemove)
|
||||
@@ -629,7 +628,7 @@ object HoodieSparkSqlWriter {
|
||||
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX.key)))
|
||||
val commitSuccess =
|
||||
client.commit(tableInstantInfo.instantTime, writeResult.getWriteStatuses,
|
||||
common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))),
|
||||
common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))),
|
||||
tableInstantInfo.commitActionType,
|
||||
writeResult.getPartitionToReplaceFileIds)
|
||||
|
||||
@@ -643,7 +642,7 @@ object HoodieSparkSqlWriter {
|
||||
val asyncCompactionEnabled = isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())
|
||||
val compactionInstant: common.util.Option[java.lang.String] =
|
||||
if (asyncCompactionEnabled) {
|
||||
client.scheduleCompaction(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
client.scheduleCompaction(common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
} else {
|
||||
common.util.Option.empty()
|
||||
}
|
||||
@@ -653,7 +652,7 @@ object HoodieSparkSqlWriter {
|
||||
val asyncClusteringEnabled = isAsyncClusteringEnabled(client, parameters)
|
||||
val clusteringInstant: common.util.Option[java.lang.String] =
|
||||
if (asyncClusteringEnabled) {
|
||||
client.scheduleClustering(common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
client.scheduleClustering(common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
} else {
|
||||
common.util.Option.empty()
|
||||
}
|
||||
|
||||
@@ -19,9 +19,11 @@ package org.apache.spark.sql.hudi
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
import org.apache.hudi.index.zorder.ZOrderingIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor}
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
|
||||
import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, EqualNullSafe, EqualTo, Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, StartsWith}
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
@@ -29,181 +31,230 @@ import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.sources.Filter
|
||||
import org.apache.spark.sql.types.{StringType, StructType}
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object DataSkippingUtils {
|
||||
object DataSkippingUtils extends Logging {
|
||||
|
||||
/**
|
||||
* Translates provided {@link filterExpr} into corresponding filter-expression for Z-index index table
|
||||
* to filter out candidate files that would hold records matching the original filter
|
||||
*
|
||||
* @param filterExpr original filter from query
|
||||
* @param sourceFilterExpr original filter from query
|
||||
* @param indexSchema index table schema
|
||||
* @return filter for Z-index table
|
||||
*/
|
||||
def createZIndexLookupFilter(filterExpr: Expression, indexSchema: StructType): Expression = {
|
||||
|
||||
def rewriteCondition(colName: Seq[String], conditionExpress: Expression): Expression = {
|
||||
val stats = Set.apply(
|
||||
UnresolvedAttribute(colName).name + "_minValue",
|
||||
UnresolvedAttribute(colName).name + "_maxValue",
|
||||
UnresolvedAttribute(colName).name + "_num_nulls"
|
||||
)
|
||||
|
||||
if (stats.forall(stat => indexSchema.exists(_.name == stat))) {
|
||||
conditionExpress
|
||||
} else {
|
||||
Literal.TrueLiteral
|
||||
}
|
||||
def createZIndexLookupFilter(sourceFilterExpr: Expression, indexSchema: StructType): Expression = {
|
||||
// Try to transform original Source Table's filter expression into
|
||||
// Column-Stats Index filter expression
|
||||
tryComposeIndexFilterExpr(sourceFilterExpr, indexSchema) match {
|
||||
case Some(e) => e
|
||||
// NOTE: In case we can't transform source filter expression, we fallback
|
||||
// to {@code TrueLiteral}, to essentially avoid pruning any indexed files from scanning
|
||||
case None => TrueLiteral
|
||||
}
|
||||
}
|
||||
|
||||
def refColExpr(colName: Seq[String], statisticValue: String): Expression =
|
||||
col(UnresolvedAttribute(colName).name + statisticValue).expr
|
||||
private def tryComposeIndexFilterExpr(sourceExpr: Expression, indexSchema: StructType): Option[Expression] = {
|
||||
def minValue(colName: String) = col(getMinColumnNameFor(colName)).expr
|
||||
def maxValue(colName: String) = col(getMaxColumnNameFor(colName)).expr
|
||||
def numNulls(colName: String) = col(getNumNullsColumnNameFor(colName)).expr
|
||||
|
||||
def minValue(colName: Seq[String]) = refColExpr(colName, "_minValue")
|
||||
def maxValue(colName: Seq[String]) = refColExpr(colName, "_maxValue")
|
||||
def numNulls(colName: Seq[String]) = refColExpr(colName, "_num_nulls")
|
||||
|
||||
def colContainsValuesEqualToLiteral(colName: Seq[String], value: Literal) =
|
||||
def colContainsValuesEqualToLiteral(colName: String, value: Literal): Expression =
|
||||
// Only case when column C contains value V is when min(C) <= V <= max(c)
|
||||
And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value))
|
||||
|
||||
def colContainsValuesEqualToLiterals(colName: Seq[String], list: Seq[Literal]) =
|
||||
list.map { lit => colContainsValuesEqualToLiteral(colName, lit) }.reduce(Or)
|
||||
def colContainsOnlyValuesEqualToLiteral(colName: String, value: Literal) =
|
||||
// Only case when column C contains _only_ value V is when min(C) = V AND max(c) = V
|
||||
And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value))
|
||||
|
||||
filterExpr match {
|
||||
sourceExpr match {
|
||||
// Filter "colA = b"
|
||||
// Translates to "colA_minValue <= b AND colA_maxValue >= b" condition for index lookup
|
||||
case EqualTo(attribute: AttributeReference, value: Literal) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, colContainsValuesEqualToLiteral(colName, value))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => colContainsValuesEqualToLiteral(colName, value))
|
||||
|
||||
// Filter "b = colA"
|
||||
// Translates to "colA_minValue <= b AND colA_maxValue >= b" condition for index lookup
|
||||
case EqualTo(value: Literal, attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, colContainsValuesEqualToLiteral(colName, value))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => colContainsValuesEqualToLiteral(colName, value))
|
||||
|
||||
// Filter "colA != b"
|
||||
// Translates to "NOT(colA_minValue = b AND colA_maxValue = b)"
|
||||
// NOTE: This is NOT an inversion of `colA = b`
|
||||
case Not(EqualTo(attribute: AttributeReference, value: Literal)) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => Not(colContainsOnlyValuesEqualToLiteral(colName, value)))
|
||||
|
||||
// Filter "b != colA"
|
||||
// Translates to "NOT(colA_minValue = b AND colA_maxValue = b)"
|
||||
// NOTE: This is NOT an inversion of `colA = b`
|
||||
case Not(EqualTo(value: Literal, attribute: AttributeReference)) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => Not(colContainsOnlyValuesEqualToLiteral(colName, value)))
|
||||
|
||||
// Filter "colA = null"
|
||||
// Translates to "colA_num_nulls = null" for index lookup
|
||||
case equalNullSafe @ EqualNullSafe(_: AttributeReference, _ @ Literal(null, _)) =>
|
||||
val colName = getTargetColNameParts(equalNullSafe.left)
|
||||
rewriteCondition(colName, EqualTo(numNulls(colName), equalNullSafe.right))
|
||||
getTargetIndexedColName(equalNullSafe.left, indexSchema)
|
||||
.map(colName => EqualTo(numNulls(colName), equalNullSafe.right))
|
||||
|
||||
// Filter "colA < b"
|
||||
// Translates to "colA_minValue < b" for index lookup
|
||||
case LessThan(attribute: AttributeReference, value: Literal) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, LessThan(minValue(colName), value))
|
||||
// Filter "b < colA"
|
||||
// Translates to "b < colA_maxValue" for index lookup
|
||||
case LessThan(value: Literal, attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, GreaterThan(maxValue(colName), value))
|
||||
// Filter "colA > b"
|
||||
// Translates to "colA_maxValue > b" for index lookup
|
||||
case GreaterThan(attribute: AttributeReference, value: Literal) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, GreaterThan(maxValue(colName), value))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThan(minValue(colName), value))
|
||||
|
||||
// Filter "b > colA"
|
||||
// Translates to "b > colA_minValue" for index lookup
|
||||
case GreaterThan(value: Literal, attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, LessThan(minValue(colName), value))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThan(minValue(colName), value))
|
||||
|
||||
// Filter "b < colA"
|
||||
// Translates to "b < colA_maxValue" for index lookup
|
||||
case LessThan(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThan(maxValue(colName), value))
|
||||
|
||||
// Filter "colA > b"
|
||||
// Translates to "colA_maxValue > b" for index lookup
|
||||
case GreaterThan(attribute: AttributeReference, value: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThan(maxValue(colName), value))
|
||||
|
||||
// Filter "colA <= b"
|
||||
// Translates to "colA_minValue <= b" for index lookup
|
||||
case LessThanOrEqual(attribute: AttributeReference, value: Literal) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, LessThanOrEqual(minValue(colName), value))
|
||||
// Filter "b <= colA"
|
||||
// Translates to "b <= colA_maxValue" for index lookup
|
||||
case LessThanOrEqual(value: Literal, attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, GreaterThanOrEqual(maxValue(colName), value))
|
||||
// Filter "colA >= b"
|
||||
// Translates to "colA_maxValue >= b" for index lookup
|
||||
case GreaterThanOrEqual(attribute: AttributeReference, right: Literal) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, GreaterThanOrEqual(maxValue(colName), right))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThanOrEqual(minValue(colName), value))
|
||||
|
||||
// Filter "b >= colA"
|
||||
// Translates to "b >= colA_minValue" for index lookup
|
||||
case GreaterThanOrEqual(value: Literal, attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, LessThanOrEqual(minValue(colName), value))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThanOrEqual(minValue(colName), value))
|
||||
|
||||
// Filter "b <= colA"
|
||||
// Translates to "b <= colA_maxValue" for index lookup
|
||||
case LessThanOrEqual(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThanOrEqual(maxValue(colName), value))
|
||||
|
||||
// Filter "colA >= b"
|
||||
// Translates to "colA_maxValue >= b" for index lookup
|
||||
case GreaterThanOrEqual(attribute: AttributeReference, right: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThanOrEqual(maxValue(colName), right))
|
||||
|
||||
// Filter "colA is null"
|
||||
// Translates to "colA_num_nulls > 0" for index lookup
|
||||
case IsNull(attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, GreaterThan(numNulls(colName), Literal(0)))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThan(numNulls(colName), Literal(0)))
|
||||
|
||||
// Filter "colA is not null"
|
||||
// Translates to "colA_num_nulls = 0" for index lookup
|
||||
case IsNotNull(attribute: AttributeReference) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, EqualTo(numNulls(colName), Literal(0)))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => EqualTo(numNulls(colName), Literal(0)))
|
||||
|
||||
// Filter "colA in (a, b, ...)"
|
||||
// Translates to "(colA_minValue <= a AND colA_maxValue >= a) OR (colA_minValue <= b AND colA_maxValue >= b)" for index lookup
|
||||
// NOTE: This is equivalent to "colA = a OR colA = b OR ..."
|
||||
case In(attribute: AttributeReference, list: Seq[Literal]) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, colContainsValuesEqualToLiterals(colName, list))
|
||||
// Filter "colA like xxx"
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName =>
|
||||
list.map { lit => colContainsValuesEqualToLiteral(colName, lit) }.reduce(Or)
|
||||
)
|
||||
|
||||
// Filter "colA not in (a, b, ...)"
|
||||
// Translates to "NOT((colA_minValue = a AND colA_maxValue = a) OR (colA_minValue = b AND colA_maxValue = b))" for index lookup
|
||||
// NOTE: This is NOT an inversion of `in (a, b, ...)` expr, this is equivalent to "colA != a AND colA != b AND ..."
|
||||
case Not(In(attribute: AttributeReference, list: Seq[Literal])) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName =>
|
||||
Not(
|
||||
list.map { lit => colContainsOnlyValuesEqualToLiteral(colName, lit) }.reduce(Or)
|
||||
)
|
||||
)
|
||||
|
||||
// Filter "colA like 'xxx%'"
|
||||
// Translates to "colA_minValue <= xxx AND colA_maxValue >= xxx" for index lookup
|
||||
// NOTE: That this operator only matches string prefixes, and this is
|
||||
// essentially equivalent to "colA = b" expression
|
||||
case StartsWith(attribute, v @ Literal(_: UTF8String, _)) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, colContainsValuesEqualToLiteral(colName, v))
|
||||
// Filter "colA not in (a, b, ...)"
|
||||
// Translates to "(colA_minValue > a OR colA_maxValue < a) AND (colA_minValue > b OR colA_maxValue < b)" for index lookup
|
||||
// NOTE: This is an inversion of `in (a, b, ...)` expr
|
||||
case Not(In(attribute: AttributeReference, list: Seq[Literal])) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, Not(colContainsValuesEqualToLiterals(colName, list)))
|
||||
// Filter "colA != b"
|
||||
// Translates to "colA_minValue > b OR colA_maxValue < b" (which is an inversion of expr for "colA = b") for index lookup
|
||||
// NOTE: This is an inversion of `colA = b` expr
|
||||
case Not(EqualTo(attribute: AttributeReference, value: Literal)) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, Not(colContainsValuesEqualToLiteral(colName, value)))
|
||||
// Filter "b != colA"
|
||||
// Translates to "colA_minValue > b OR colA_maxValue < b" (which is an inversion of expr for "colA = b") for index lookup
|
||||
// NOTE: This is an inversion of `colA != b` expr
|
||||
case Not(EqualTo(value: Literal, attribute: AttributeReference)) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, Not(colContainsValuesEqualToLiteral(colName, value)))
|
||||
// Filter "colA not like xxx"
|
||||
// Translates to "!(colA_minValue <= xxx AND colA_maxValue >= xxx)" for index lookup
|
||||
// NOTE: This is a inversion of "colA like xxx" assuming that colA is a string-based type
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => colContainsValuesEqualToLiteral(colName, v))
|
||||
|
||||
// Filter "colA not like 'xxx%'"
|
||||
// Translates to "NOT(colA_minValue like 'xxx%' AND colA_maxValue like 'xxx%')" for index lookup
|
||||
// NOTE: This is NOT an inversion of "colA like xxx"
|
||||
case Not(StartsWith(attribute, value @ Literal(_: UTF8String, _))) =>
|
||||
val colName = getTargetColNameParts(attribute)
|
||||
rewriteCondition(colName, Not(colContainsValuesEqualToLiteral(colName, value)))
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName =>
|
||||
Not(And(StartsWith(minValue(colName), value), StartsWith(maxValue(colName), value)))
|
||||
)
|
||||
|
||||
case or: Or =>
|
||||
val resLeft = createZIndexLookupFilter(or.left, indexSchema)
|
||||
val resRight = createZIndexLookupFilter(or.right, indexSchema)
|
||||
Or(resLeft, resRight)
|
||||
|
||||
Option(Or(resLeft, resRight))
|
||||
|
||||
case and: And =>
|
||||
val resLeft = createZIndexLookupFilter(and.left, indexSchema)
|
||||
val resRight = createZIndexLookupFilter(and.right, indexSchema)
|
||||
And(resLeft, resRight)
|
||||
|
||||
case expr: Expression =>
|
||||
Literal.TrueLiteral
|
||||
Option(And(resLeft, resRight))
|
||||
|
||||
//
|
||||
// Pushing Logical NOT inside the AND/OR expressions
|
||||
// NOTE: This is required to make sure we're properly handling negations in
|
||||
// cases like {@code NOT(colA = 0)}, {@code NOT(colA in (a, b, ...)}
|
||||
//
|
||||
|
||||
case Not(And(left: Expression, right: Expression)) =>
|
||||
Option(createZIndexLookupFilter(Or(Not(left), Not(right)), indexSchema))
|
||||
|
||||
case Not(Or(left: Expression, right: Expression)) =>
|
||||
Option(createZIndexLookupFilter(And(Not(left), Not(right)), indexSchema))
|
||||
|
||||
case _: Expression => None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts name from a resolved expression referring to a nested or non-nested column.
|
||||
*/
|
||||
def getTargetColNameParts(resolvedTargetCol: Expression): Seq[String] = {
|
||||
private def checkColIsIndexed(colName: String, indexSchema: StructType): Boolean = {
|
||||
Set.apply(
|
||||
getMinColumnNameFor(colName),
|
||||
getMaxColumnNameFor(colName),
|
||||
getNumNullsColumnNameFor(colName)
|
||||
)
|
||||
.forall(stat => indexSchema.exists(_.name == stat))
|
||||
}
|
||||
|
||||
private def getTargetIndexedColName(resolvedExpr: Expression, indexSchema: StructType): Option[String] = {
|
||||
val colName = UnresolvedAttribute(getTargetColNameParts(resolvedExpr)).name
|
||||
|
||||
// Verify that the column is indexed
|
||||
if (checkColIsIndexed(colName, indexSchema)) {
|
||||
Option.apply(colName)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
private def getTargetColNameParts(resolvedTargetCol: Expression): Seq[String] = {
|
||||
resolvedTargetCol match {
|
||||
case attr: Attribute => Seq(attr.name)
|
||||
|
||||
case Alias(c, _) => getTargetColNameParts(c)
|
||||
|
||||
case GetStructField(c, _, Some(name)) => getTargetColNameParts(c) :+ name
|
||||
|
||||
case ex: ExtractValue =>
|
||||
throw new AnalysisException(s"convert reference to name failed, Updating nested fields is only supported for StructType: ${ex}.")
|
||||
|
||||
case other =>
|
||||
throw new AnalysisException(s"convert reference to name failed, Found unsupported expression ${other}")
|
||||
}
|
||||
|
||||
@@ -0,0 +1,8 @@
|
||||
{"c1_maxValue":1000,"c1_minValue":3,"c1_num_nulls":0,"c2_maxValue":" 993sdc","c2_minValue":" 1000sdc","c2_num_nulls":0,"c3_maxValue":999.348,"c3_minValue":5.102,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-27","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"}
|
||||
{"c1_maxValue":1000,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 996sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":999.779,"c3_minValue":2.992,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/g==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
{"c1_maxValue":998,"c1_minValue":2,"c1_num_nulls":0,"c2_maxValue":" 998sdc","c2_minValue":" 104sdc","c2_num_nulls":0,"c3_maxValue":997.905,"c3_minValue":0.876,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-02","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"Ag==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"}
|
||||
{"c1_maxValue":997,"c1_minValue":3,"c1_num_nulls":0,"c2_maxValue":" 9sdc","c2_minValue":" 102sdc","c2_num_nulls":0,"c3_maxValue":990.531,"c3_minValue":2.336,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-27","c6_minValue":"2020-01-02","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
{"c1_maxValue":994,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 9sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":997.496,"c3_minValue":7.742,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"}
|
||||
{"c1_maxValue":999,"c1_minValue":1,"c1_num_nulls":0,"c2_maxValue":" 999sdc","c2_minValue":" 100sdc","c2_num_nulls":0,"c3_maxValue":980.676,"c3_minValue":0.120,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-03","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
{"c1_maxValue":999,"c1_minValue":1,"c1_num_nulls":0,"c2_maxValue":" 99sdc","c2_minValue":" 10sdc","c2_num_nulls":0,"c3_maxValue":993.940,"c3_minValue":4.598,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-03","c6_num_nulls":0,"c7_maxValue":"/g==","c7_minValue":"AQ==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-1c8226c2-f2a0-455d-aedd-c544003b0b3d-c000.snappy.parquet"}
|
||||
{"c1_maxValue":998,"c1_minValue":6,"c1_num_nulls":0,"c2_maxValue":" 99sdc","c2_minValue":" 111sdc","c2_num_nulls":0,"c3_maxValue":999.282,"c3_minValue":1.217,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":2,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
@@ -0,0 +1,4 @@
|
||||
{"c1_maxValue":1000,"c1_minValue":0,"c1_num_nulls":0,"c2_maxValue":" 996sdc","c2_minValue":" 0sdc","c2_num_nulls":0,"c3_maxValue":999.779,"c3_minValue":2.992,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/g==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00000-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
{"c1_maxValue":997,"c1_minValue":3,"c1_num_nulls":0,"c2_maxValue":" 9sdc","c2_minValue":" 102sdc","c2_num_nulls":0,"c3_maxValue":990.531,"c3_minValue":2.336,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-27","c6_minValue":"2020-01-02","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00001-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
{"c1_maxValue":999,"c1_minValue":1,"c1_num_nulls":0,"c2_maxValue":" 999sdc","c2_minValue":" 100sdc","c2_num_nulls":0,"c3_maxValue":980.676,"c3_minValue":0.120,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":1,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-03","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00002-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
{"c1_maxValue":998,"c1_minValue":6,"c1_num_nulls":0,"c2_maxValue":" 99sdc","c2_minValue":" 111sdc","c2_num_nulls":0,"c3_maxValue":999.282,"c3_minValue":1.217,"c3_num_nulls":0,"c5_maxValue":101,"c5_minValue":2,"c5_num_nulls":0,"c6_maxValue":"2020-11-28","c6_minValue":"2020-01-01","c6_num_nulls":0,"c7_maxValue":"/w==","c7_minValue":"AA==","c7_num_nulls":0,"c8_maxValue":9,"c8_minValue":9,"c8_num_nulls":0,"file":"part-00003-5034d84a-c4c8-4eba-85b5-a52f47e628a7-c000.snappy.parquet"}
|
||||
@@ -46,7 +46,6 @@ import org.mockito.Mockito.{spy, times, verify}
|
||||
import org.scalatest.Matchers.{assertResult, be, convertToAnyShouldWrapper, intercept}
|
||||
|
||||
import java.time.Instant
|
||||
import java.util
|
||||
import java.util.{Collections, Date, UUID}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
@@ -147,7 +146,7 @@ class HoodieSparkSqlWriterSuite {
|
||||
* @param inputList list of Row
|
||||
* @return list of Seq
|
||||
*/
|
||||
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
|
||||
def convertRowListToSeq(inputList: java.util.List[Row]): Seq[Row] =
|
||||
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
|
||||
|
||||
/**
|
||||
|
||||
@@ -0,0 +1,365 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.hudi.index.zorder.ZOrderingIndexHelper
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, Not}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation}
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.hudi.DataSkippingUtils
|
||||
import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType, VarcharType}
|
||||
import org.apache.spark.sql.{Column, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.Arguments.arguments
|
||||
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
// NOTE: Only A, B columns are indexed
|
||||
case class IndexRow(
|
||||
file: String,
|
||||
A_minValue: Long,
|
||||
A_maxValue: Long,
|
||||
A_num_nulls: Long,
|
||||
B_minValue: String = null,
|
||||
B_maxValue: String = null,
|
||||
B_num_nulls: Long = -1
|
||||
)
|
||||
|
||||
class TestDataSkippingUtils extends HoodieClientTestBase {
|
||||
|
||||
var spark: SparkSession = _
|
||||
|
||||
@BeforeEach
|
||||
override def setUp(): Unit = {
|
||||
initSparkContexts()
|
||||
spark = sqlContext.sparkSession
|
||||
}
|
||||
|
||||
val indexedCols = Seq("A", "B")
|
||||
val sourceTableSchema =
|
||||
StructType(
|
||||
Seq(
|
||||
StructField("A", LongType),
|
||||
StructField("B", StringType),
|
||||
StructField("C", VarcharType(32))
|
||||
)
|
||||
)
|
||||
|
||||
val indexSchema =
|
||||
ZOrderingIndexHelper.composeIndexSchema(
|
||||
sourceTableSchema.fields.toSeq
|
||||
.filter(f => indexedCols.contains(f.name))
|
||||
.asJava
|
||||
)
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("testBaseLookupFilterExpressionsSource", "testAdvancedLookupFilterExpressionsSource"))
|
||||
def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = {
|
||||
val resolvedExpr: Expression = resolveFilterExpr(sourceExpr, sourceTableSchema)
|
||||
|
||||
val lookupFilter = DataSkippingUtils.createZIndexLookupFilter(resolvedExpr, indexSchema)
|
||||
|
||||
val spark2 = spark
|
||||
import spark2.implicits._
|
||||
|
||||
val indexDf = spark.createDataset(input)
|
||||
|
||||
val rows = indexDf.where(new Column(lookupFilter))
|
||||
.select("file")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSeq
|
||||
|
||||
assertEquals(output, rows)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@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 spark2 = spark
|
||||
import spark2.implicits._
|
||||
|
||||
val indexDf = spark.createDataset(input)
|
||||
|
||||
val rows = indexDf.where(new Column(lookupFilter))
|
||||
.select("file")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSeq
|
||||
|
||||
assertEquals(output, rows)
|
||||
}
|
||||
|
||||
private def resolveFilterExpr(exprString: String, tableSchema: StructType): Expression = {
|
||||
val expr = spark.sessionState.sqlParser.parseExpression(exprString)
|
||||
resolveFilterExpr(expr, tableSchema)
|
||||
}
|
||||
|
||||
private def resolveFilterExpr(expr: Expression, tableSchema: StructType): Expression = {
|
||||
val schemaFields = tableSchema.fields
|
||||
val resolvedExpr = spark.sessionState.analyzer.ResolveReferences(
|
||||
Filter(expr, LocalRelation(schemaFields.head, schemaFields.drop(1): _*))
|
||||
)
|
||||
.asInstanceOf[Filter].condition
|
||||
|
||||
checkForUnresolvedRefs(resolvedExpr)
|
||||
}
|
||||
|
||||
def checkForUnresolvedRefs(resolvedExpr: Expression): Expression =
|
||||
resolvedExpr match {
|
||||
case UnresolvedAttribute(_) => throw new IllegalStateException("unresolved attribute")
|
||||
case _ => resolvedExpr.mapChildren(e => checkForUnresolvedRefs(e))
|
||||
}
|
||||
}
|
||||
|
||||
object TestDataSkippingUtils {
|
||||
def testStringsLookupFilterExpressionsSource(): java.util.stream.Stream[Arguments] = {
|
||||
java.util.stream.Stream.of(
|
||||
arguments(
|
||||
col("B").startsWith("abc").expr,
|
||||
Seq(
|
||||
IndexRow("file_1", 0, 0, 0, "aba", "adf", 1), // may contain strings starting w/ "abc"
|
||||
IndexRow("file_2", 0, 0, 0, "adf", "azy", 0),
|
||||
IndexRow("file_3", 0, 0, 0, "aaa", "aba", 0)
|
||||
),
|
||||
Seq("file_1")),
|
||||
arguments(
|
||||
Not(col("B").startsWith("abc").expr),
|
||||
Seq(
|
||||
IndexRow("file_1", 0, 0, 0, "aba", "adf", 1), // may contain strings starting w/ "abc"
|
||||
IndexRow("file_2", 0, 0, 0, "adf", "azy", 0),
|
||||
IndexRow("file_3", 0, 0, 0, "aaa", "aba", 0),
|
||||
IndexRow("file_4", 0, 0, 0, "abc123", "abc345", 0) // all strings start w/ "abc"
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3"))
|
||||
)
|
||||
}
|
||||
|
||||
def testBaseLookupFilterExpressionsSource(): java.util.stream.Stream[Arguments] = {
|
||||
java.util.stream.Stream.of(
|
||||
// TODO cases
|
||||
// A = null
|
||||
arguments(
|
||||
"A = 0",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0)
|
||||
),
|
||||
Seq("file_2")),
|
||||
arguments(
|
||||
"0 = A",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0)
|
||||
),
|
||||
Seq("file_2")),
|
||||
arguments(
|
||||
"A != 0",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", 0, 0, 0) // Contains only 0s
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"0 != A",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", 0, 0, 0) // Contains only 0s
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"A < 0",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_2", "file_3")),
|
||||
arguments(
|
||||
"0 > A",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_2", "file_3")),
|
||||
arguments(
|
||||
"A > 0",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"0 < A",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"A <= -1",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_2", "file_3")),
|
||||
arguments(
|
||||
"-1 >= A",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_2", "file_3")),
|
||||
arguments(
|
||||
"A >= 1",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"1 <= A",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"A is null",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 1)
|
||||
),
|
||||
Seq("file_2")),
|
||||
arguments(
|
||||
"A is not null",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 1)
|
||||
),
|
||||
Seq("file_1")),
|
||||
arguments(
|
||||
"A in (0, 1)",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0)
|
||||
),
|
||||
Seq("file_1", "file_2")),
|
||||
arguments(
|
||||
"A not in (0, 1)",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0), // only contains 0
|
||||
IndexRow("file_5", 1, 1, 0) // only contains 1
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3"))
|
||||
)
|
||||
}
|
||||
|
||||
def testAdvancedLookupFilterExpressionsSource(): java.util.stream.Stream[Arguments] = {
|
||||
java.util.stream.Stream.of(
|
||||
arguments(
|
||||
// Filter out all rows that contain either A = 0 OR A = 1
|
||||
"A != 0 AND A != 1",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0), // only contains 0
|
||||
IndexRow("file_5", 1, 1, 0) // only contains 1
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3")),
|
||||
arguments(
|
||||
// This is an equivalent to the above expression
|
||||
"NOT(A = 0 OR A = 1)",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0), // only contains 0
|
||||
IndexRow("file_5", 1, 1, 0) // only contains 1
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3")),
|
||||
|
||||
arguments(
|
||||
// Filter out all rows that contain A = 0 AND B = 'abc'
|
||||
"A != 0 OR B != 'abc'",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0, "abc", "abc", 0), // only contains A = 0, B = 'abc'
|
||||
IndexRow("file_5", 0, 0, 0, "abc", "abc", 0) // only contains A = 0, B = 'abc'
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3")),
|
||||
arguments(
|
||||
// This is an equivalent to the above expression
|
||||
"NOT(A = 0 AND B = 'abc')",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0, "abc", "abc", 0), // only contains A = 0, B = 'abc'
|
||||
IndexRow("file_5", 0, 0, 0, "abc", "abc", 0) // only contains A = 0, B = 'abc'
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3")),
|
||||
|
||||
arguments(
|
||||
// Queries contains expression involving non-indexed column C
|
||||
"A = 0 AND B = 'abc' AND C = '...'",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0, "aaa", "xyz", 0) // might contain A = 0 AND B = 'abc'
|
||||
),
|
||||
Seq("file_4")),
|
||||
|
||||
arguments(
|
||||
// Queries contains expression involving non-indexed column C
|
||||
"A = 0 OR B = 'abc' OR C = '...'",
|
||||
Seq(
|
||||
IndexRow("file_1", 1, 2, 0),
|
||||
IndexRow("file_2", -1, 1, 0),
|
||||
IndexRow("file_3", -2, -1, 0),
|
||||
IndexRow("file_4", 0, 0, 0, "aaa", "xyz", 0) // might contain B = 'abc'
|
||||
),
|
||||
Seq("file_1", "file_2", "file_3", "file_4"))
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -59,7 +59,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
DataSourceReadOptions.QUERY_TYPE.key -> DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
@BeforeEach
|
||||
override def setUp() {
|
||||
setTableName("hoodie_test")
|
||||
initPath()
|
||||
initSparkContexts()
|
||||
|
||||
@@ -19,20 +19,17 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
|
||||
import java.io.File
|
||||
import java.nio.file.Paths
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
import org.apache.spark.sql.avro.IncompatibleSchemaException
|
||||
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull, assertTrue, fail}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.io.TempDir
|
||||
|
||||
import java.util
|
||||
import java.io.File
|
||||
import java.nio.file.Paths
|
||||
import scala.collection.JavaConverters
|
||||
|
||||
class TestHoodieSparkUtils {
|
||||
@@ -235,6 +232,6 @@ class TestHoodieSparkUtils {
|
||||
spark.stop()
|
||||
}
|
||||
|
||||
def convertRowListToSeq(inputList: util.List[Row]): Seq[Row] =
|
||||
def convertRowListToSeq(inputList: java.util.List[Row]): Seq[Row] =
|
||||
JavaConverters.asScalaIteratorConverter(inputList.iterator).asScala.toSeq
|
||||
}
|
||||
|
||||
@@ -1,231 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
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, 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
|
||||
|
||||
@Tag("functional")
|
||||
class TestTableLayoutOptimization extends HoodieClientTestBase {
|
||||
var spark: SparkSession = _
|
||||
|
||||
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()
|
||||
}
|
||||
|
||||
@AfterEach override def tearDown() = {
|
||||
cleanupSparkContexts()
|
||||
cleanupTestDataGenerator()
|
||||
cleanupFileSystem()
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ"))
|
||||
def testOptimizeWithClustering(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 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
|
||||
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 sort for all primitive type, should not throw exception.
|
||||
ZCurveOptimizeHelper.createZIndexedDataFrameByMapValue(df, "c1,c2,c3,c4,c5,c6,c7,c8", 20).show(1)
|
||||
ZCurveOptimizeHelper.createZIndexedDataFrameBySample(df, "c1,c2,c3,c4,c5,c6,c7,c8", 20).show(1)
|
||||
// do not support TimeStampType, so if we collect statistics for c4, should throw exception
|
||||
val colDf = ZCurveOptimizeHelper.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
|
||||
ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "2", Seq("0", "1"))
|
||||
// save again
|
||||
ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "3", Seq("0", "1", "2"))
|
||||
// test old index table clean
|
||||
ZCurveOptimizeHelper.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.
|
||||
ZCurveOptimizeHelper.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.getMinValueAsString, res.getMaxValueAsString)
|
||||
}
|
||||
|
||||
// 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.getMinValueAsString, res.getMaxValueAsString)
|
||||
}
|
||||
|
||||
// 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"
|
||||
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, schema)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,398 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
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.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 java.sql.{Date, Timestamp}
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.util.Random
|
||||
|
||||
@Tag("functional")
|
||||
class TestZOrderLayoutOptimization 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",
|
||||
"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()
|
||||
}
|
||||
|
||||
@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")
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
def testZIndexTableComposition(): Unit = {
|
||||
val inputDf =
|
||||
// NOTE: Schema here is provided for validation that the input date is in the appropriate format
|
||||
spark.read
|
||||
.schema(sourceTableSchema)
|
||||
.parquet(
|
||||
getClass.getClassLoader.getResource("index/zorder/input-table").toString
|
||||
)
|
||||
|
||||
val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8")
|
||||
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 =
|
||||
ZOrderingIndexHelper.buildZIndexTableFor(
|
||||
inputDf.sparkSession,
|
||||
inputDf.inputFiles.toSeq,
|
||||
zorderedColsSchemaFields
|
||||
)
|
||||
|
||||
val indexSchema =
|
||||
ZOrderingIndexHelper.composeIndexSchema(
|
||||
sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq
|
||||
)
|
||||
|
||||
// Collect Z-index stats manually (reading individual Parquet files)
|
||||
val manualZIndexTableDf =
|
||||
buildZIndexTableManually(
|
||||
getClass.getClassLoader.getResource("index/zorder/input-table").toString,
|
||||
zorderedCols,
|
||||
indexSchema
|
||||
)
|
||||
|
||||
// NOTE: Z-index is built against stats collected w/in Parquet footers, which will be
|
||||
// represented w/ corresponding Parquet schema (INT, INT64, INT96, etc).
|
||||
//
|
||||
// When stats are collected manually, produced Z-index table is inherently coerced into the
|
||||
// schema of the original source Parquet base-file and therefore we have to similarly coerce newly
|
||||
// built Z-index table (built off Parquet footers) into the canonical index schema (built off the
|
||||
// original source file schema)
|
||||
assertEquals(asJson(sort(manualZIndexTableDf)), asJson(sort(newZIndexTableDf)))
|
||||
|
||||
// Match against expected Z-index table
|
||||
val expectedZIndexTableDf =
|
||||
spark.read
|
||||
.schema(indexSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString)
|
||||
|
||||
assertEquals(asJson(sort(expectedZIndexTableDf)), asJson(sort(newZIndexTableDf)))
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
def testZIndexTableMerge(): Unit = {
|
||||
val testZIndexPath = new Path(basePath, "zindex")
|
||||
|
||||
val zorderedCols = Seq("c1", "c2", "c3", "c5", "c6", "c7", "c8")
|
||||
val indexSchema =
|
||||
ZOrderingIndexHelper.composeIndexSchema(
|
||||
sourceTableSchema.fields.filter(f => zorderedCols.contains(f.name)).toSeq
|
||||
)
|
||||
|
||||
//
|
||||
// Bootstrap Z-index table
|
||||
//
|
||||
|
||||
val firstCommitInstance = "0"
|
||||
val firstInputDf =
|
||||
spark.read.parquet(
|
||||
getClass.getClassLoader.getResource("index/zorder/input-table").toString
|
||||
)
|
||||
|
||||
ZOrderingIndexHelper.updateZIndexFor(
|
||||
firstInputDf.sparkSession,
|
||||
sourceTableSchema,
|
||||
firstInputDf.inputFiles.toSeq,
|
||||
zorderedCols.toSeq,
|
||||
testZIndexPath.toString,
|
||||
firstCommitInstance,
|
||||
Seq()
|
||||
)
|
||||
|
||||
// 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)
|
||||
|
||||
val expectedInitialZIndexTableDf =
|
||||
spark.read
|
||||
.schema(indexSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table.json").toString)
|
||||
|
||||
assertEquals(asJson(sort(expectedInitialZIndexTableDf)), asJson(sort(initialZIndexTable)))
|
||||
|
||||
val secondCommitInstance = "1"
|
||||
val secondInputDf =
|
||||
spark.read
|
||||
.schema(sourceTableSchema)
|
||||
.parquet(
|
||||
getClass.getClassLoader.getResource("index/zorder/another-input-table").toString
|
||||
)
|
||||
|
||||
//
|
||||
// Update Z-index table
|
||||
//
|
||||
|
||||
ZOrderingIndexHelper.updateZIndexFor(
|
||||
secondInputDf.sparkSession,
|
||||
sourceTableSchema,
|
||||
secondInputDf.inputFiles.toSeq,
|
||||
zorderedCols.toSeq,
|
||||
testZIndexPath.toString,
|
||||
secondCommitInstance,
|
||||
Seq(firstCommitInstance)
|
||||
)
|
||||
|
||||
// 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)
|
||||
|
||||
val expectedMergedZIndexTableDf =
|
||||
spark.read
|
||||
.schema(indexSchema)
|
||||
.json(getClass.getClassLoader.getResource("index/zorder/z-index-table-merged.json").toString)
|
||||
|
||||
assertEquals(asJson(sort(expectedMergedZIndexTableDf)), asJson(sort(mergedZIndexTable)))
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
def testZIndexTablesGarbageCollection(): Unit = {
|
||||
val testZIndexPath = new Path(System.getProperty("java.io.tmpdir"), "zindex")
|
||||
val fs = testZIndexPath.getFileSystem(spark.sparkContext.hadoopConfiguration)
|
||||
|
||||
val inputDf =
|
||||
spark.read.parquet(
|
||||
getClass.getClassLoader.getResource("index/zorder/input-table").toString
|
||||
)
|
||||
|
||||
// Try to save statistics
|
||||
ZOrderingIndexHelper.updateZIndexFor(
|
||||
inputDf.sparkSession,
|
||||
sourceTableSchema,
|
||||
inputDf.inputFiles.toSeq,
|
||||
Seq("c1","c2","c3","c5","c6","c7","c8"),
|
||||
testZIndexPath.toString,
|
||||
"2",
|
||||
Seq("0", "1")
|
||||
)
|
||||
|
||||
// Save again
|
||||
ZOrderingIndexHelper.updateZIndexFor(
|
||||
inputDf.sparkSession,
|
||||
sourceTableSchema,
|
||||
inputDf.inputFiles.toSeq,
|
||||
Seq("c1","c2","c3","c5","c6","c7","c8"),
|
||||
testZIndexPath.toString,
|
||||
"3",
|
||||
Seq("0", "1", "2")
|
||||
)
|
||||
|
||||
// Test old index table being cleaned up
|
||||
ZOrderingIndexHelper.updateZIndexFor(
|
||||
inputDf.sparkSession,
|
||||
sourceTableSchema,
|
||||
inputDf.inputFiles.toSeq,
|
||||
Seq("c1","c2","c3","c5","c6","c7","c8"),
|
||||
testZIndexPath.toString,
|
||||
"4",
|
||||
Seq("0", "1", "3")
|
||||
)
|
||||
|
||||
assertEquals(!fs.exists(new Path(testZIndexPath, "2")), true)
|
||||
assertEquals(!fs.exists(new Path(testZIndexPath, "3")), true)
|
||||
assertEquals(fs.exists(new Path(testZIndexPath, "4")), true)
|
||||
}
|
||||
|
||||
private def buildZIndexTableManually(tablePath: String, zorderedCols: Seq[String], indexSchema: StructType) = {
|
||||
val files = {
|
||||
val it = fs.listFiles(new Path(tablePath), true)
|
||||
var seq = Seq[LocatedFileStatus]()
|
||||
while (it.hasNext) {
|
||||
seq = seq :+ it.next()
|
||||
}
|
||||
seq
|
||||
}
|
||||
|
||||
spark.createDataFrame(
|
||||
files.flatMap(file => {
|
||||
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.selectExpr(exprs: _*)
|
||||
.collect()
|
||||
}),
|
||||
indexSchema
|
||||
)
|
||||
}
|
||||
|
||||
private def asJson(df: DataFrame) =
|
||||
df.toJSON
|
||||
.select("value")
|
||||
.collect()
|
||||
.toSeq
|
||||
.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
|
||||
// of their names, we have to shuffle new Z-index table columns order to match
|
||||
// Rows are sorted by filename as well to avoid
|
||||
val sortedCols = df.columns.sorted
|
||||
df.select(sortedCols.head, sortedCols.tail: _*)
|
||||
.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)
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user