1
0

[HUDI-3594] Supporting Composite Expressions over Data Table Columns in Data Skipping flow (#4996)

This commit is contained in:
Alexey Kudinkin
2022-03-24 22:27:15 -07:00
committed by GitHub
parent 8896864d7b
commit 8b38ddedc2
18 changed files with 1079 additions and 302 deletions

View File

@@ -72,6 +72,7 @@ import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
import static org.apache.hudi.util.DataTypeUtils.areCompatible;
public class ColumnStatsIndexHelper {
@@ -111,17 +112,17 @@ public class ColumnStatsIndexHelper {
* | another_base_file.parquet | -10 | 0 | 5 |
* +---------------------------+------------+------------+-------------+
* </pre>
*
* <p>
* NOTE: Currently {@link TimestampType} is not supported, since Parquet writer
* does not support statistics for it.
*
* <p>
* TODO leverage metadata table after RFC-27 lands
* @VisibleForTesting
*
* @param sparkSession encompassing Spark session
* @param baseFilesPaths list of base-files paths to be sourced for column-stats index
* @param sparkSession encompassing Spark session
* @param baseFilesPaths list of base-files paths to be sourced for column-stats index
* @param orderedColumnSchemas target ordered columns
* @return Spark's {@link Dataset} holding an index table
* @VisibleForTesting
*/
@Nonnull
public static Dataset<Row> buildColumnStatsTableFor(
@@ -223,13 +224,13 @@ public class ColumnStatsIndexHelper {
* <li>Cleans up any residual index tables, that weren't cleaned up before</li>
* </ol>
*
* @param sparkSession encompassing Spark session
* @param sparkSession encompassing Spark session
* @param sourceTableSchema instance of {@link StructType} bearing source table's writer's schema
* @param sourceBaseFiles list of base-files to be indexed
* @param orderedCols target ordered columns
* @param indexFolderPath col-stats index folder path
* @param commitTime current operation commit instant
* @param completedCommits all previously completed commit instants
* @param sourceBaseFiles list of base-files to be indexed
* @param orderedCols target ordered columns
* @param indexFolderPath col-stats index folder path
* @param commitTime current operation commit instant
* @param completedCommits all previously completed commit instants
*/
public static void updateColumnStatsIndexFor(
@Nonnull SparkSession sparkSession,
@@ -424,57 +425,64 @@ public class ColumnStatsIndexHelper {
return String.format("%s_%s", col, statName);
}
private static Pair<Object, Object>
fetchMinMaxValues(
@Nonnull DataType colType,
@Nonnull HoodieColumnRangeMetadata<Comparable> colMetadata) {
private static Pair<Object, Object> fetchMinMaxValues(@Nonnull DataType colType,
@Nonnull HoodieColumnRangeMetadata<Comparable> colMetadata) {
Comparable<?> minValue = colMetadata.getMinValue();
Comparable<?> maxValue = colMetadata.getMaxValue();
checkState((minValue == null) == (maxValue == null), "Either both min/max values should be null or neither");
if (minValue == null || maxValue == null) {
return Pair.of(null, null);
}
if (colType instanceof IntegerType) {
return Pair.of(
new Integer(colMetadata.getMinValue().toString()),
new Integer(colMetadata.getMaxValue().toString())
new Integer(minValue.toString()),
new Integer(maxValue.toString())
);
} else if (colType instanceof DoubleType) {
return Pair.of(
new Double(colMetadata.getMinValue().toString()),
new Double(colMetadata.getMaxValue().toString())
new Double(minValue.toString()),
new Double(maxValue.toString())
);
} else if (colType instanceof StringType) {
return Pair.of(
colMetadata.getMinValue().toString(),
colMetadata.getMaxValue().toString());
minValue.toString(),
maxValue.toString());
} else if (colType instanceof DecimalType) {
return Pair.of(
new BigDecimal(colMetadata.getMinValue().toString()),
new BigDecimal(colMetadata.getMaxValue().toString()));
new BigDecimal(minValue.toString()),
new BigDecimal(maxValue.toString()));
} else if (colType instanceof DateType) {
return Pair.of(
java.sql.Date.valueOf(colMetadata.getMinValue().toString()),
java.sql.Date.valueOf(colMetadata.getMaxValue().toString()));
java.sql.Date.valueOf(minValue.toString()),
java.sql.Date.valueOf(maxValue.toString()));
} else if (colType instanceof LongType) {
return Pair.of(
new Long(colMetadata.getMinValue().toString()),
new Long(colMetadata.getMaxValue().toString()));
new Long(minValue.toString()),
new Long(maxValue.toString()));
} else if (colType instanceof ShortType) {
return Pair.of(
new Short(colMetadata.getMinValue().toString()),
new Short(colMetadata.getMaxValue().toString()));
new Short(minValue.toString()),
new Short(maxValue.toString()));
} else if (colType instanceof FloatType) {
return Pair.of(
new Float(colMetadata.getMinValue().toString()),
new Float(colMetadata.getMaxValue().toString()));
new Float(minValue.toString()),
new Float(maxValue.toString()));
} else if (colType instanceof BinaryType) {
return Pair.of(
((ByteBuffer) colMetadata.getMinValue()).array(),
((ByteBuffer) colMetadata.getMaxValue()).array());
((ByteBuffer) minValue).array(),
((ByteBuffer) maxValue).array());
} else if (colType instanceof BooleanType) {
return Pair.of(
Boolean.valueOf(colMetadata.getMinValue().toString()),
Boolean.valueOf(colMetadata.getMaxValue().toString()));
Boolean.valueOf(minValue.toString()),
Boolean.valueOf(maxValue.toString()));
} else if (colType instanceof ByteType) {
return Pair.of(
Byte.valueOf(colMetadata.getMinValue().toString()),
Byte.valueOf(colMetadata.getMaxValue().toString()));
} else {
Byte.valueOf(minValue.toString()),
Byte.valueOf(maxValue.toString()));
} else {
throw new HoodieException(String.format("Not support type: %s", colType));
}
}

View File

@@ -27,10 +27,10 @@ import org.apache.spark.sql.hudi.SparkAdapter
trait SparkAdapterSupport {
lazy val sparkAdapter: SparkAdapter = {
val adapterClass = if (HoodieSparkUtils.gteqSpark3_2) {
val adapterClass = if (HoodieSparkUtils.isSpark3_2) {
"org.apache.spark.sql.adapter.Spark3_2Adapter"
} else if (HoodieSparkUtils.isSpark3_0 || HoodieSparkUtils.isSpark3_1) {
"org.apache.spark.sql.adapter.Spark3Adapter"
"org.apache.spark.sql.adapter.Spark3_1Adapter"
} else {
"org.apache.spark.sql.adapter.Spark2Adapter"
}

View File

@@ -0,0 +1,127 @@
/*
* 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.spark.sql
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction}
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, SubqueryExpression}
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan}
import org.apache.spark.sql.types.StructType
trait HoodieCatalystExpressionUtils {
/**
* Parses and resolves expression against the attributes of the given table schema.
*
* For example:
* <pre>
* ts > 1000 and ts <= 1500
* </pre>
* will be resolved as
* <pre>
* And(GreaterThan(ts#590L > 1000), LessThanOrEqual(ts#590L <= 1500))
* </pre>
*
* Where <pre>ts</pre> is a column of the provided [[tableSchema]]
*
* @param spark spark session
* @param exprString string representation of the expression to parse and resolve
* @param tableSchema table schema encompassing attributes to resolve against
* @return Resolved filter expression
*/
def resolveExpr(spark: SparkSession, exprString: String, tableSchema: StructType): Expression = {
val expr = spark.sessionState.sqlParser.parseExpression(exprString)
resolveExpr(spark, expr, tableSchema)
}
/**
* Resolves provided expression (unless already resolved) against the attributes of the given table schema.
*
* For example:
* <pre>
* ts > 1000 and ts <= 1500
* </pre>
* will be resolved as
* <pre>
* And(GreaterThan(ts#590L > 1000), LessThanOrEqual(ts#590L <= 1500))
* </pre>
*
* Where <pre>ts</pre> is a column of the provided [[tableSchema]]
*
* @param spark spark session
* @param expr Catalyst expression to be resolved (if not yet)
* @param tableSchema table schema encompassing attributes to resolve against
* @return Resolved filter expression
*/
def resolveExpr(spark: SparkSession, expr: Expression, tableSchema: StructType): Expression = {
val analyzer = spark.sessionState.analyzer
val schemaFields = tableSchema.fields
val resolvedExpr = {
val plan: LogicalPlan = Filter(expr, LocalRelation(schemaFields.head, schemaFields.drop(1): _*))
analyzer.execute(plan).asInstanceOf[Filter].condition
}
if (!hasUnresolvedRefs(resolvedExpr)) {
resolvedExpr
} else {
throw new IllegalStateException("unresolved attribute")
}
}
/**
* Split the given predicates into two sequence predicates:
* - predicates that references partition columns only(and involves no sub-query);
* - other predicates.
*
* @param sparkSession The spark session
* @param predicates The predicates to be split
* @param partitionColumns The partition columns
* @return (partitionFilters, dataFilters)
*/
def splitPartitionAndDataPredicates(sparkSession: SparkSession,
predicates: Array[Expression],
partitionColumns: Array[String]): (Array[Expression], Array[Expression]) = {
// Validates that the provided names both resolve to the same entity
val resolvedNameEquals = sparkSession.sessionState.analyzer.resolver
predicates.partition(expr => {
// Checks whether given expression only references partition columns(and involves no sub-query)
expr.references.forall(r => partitionColumns.exists(resolvedNameEquals(r.name, _))) &&
!SubqueryExpression.hasSubquery(expr)
})
}
/**
* Matches an expression iff
*
* <ol>
* <li>It references exactly one [[AttributeReference]]</li>
* <li>It contains only whitelisted transformations that preserve ordering of the source column [1]</li>
* </ol>
*
* [1] Preserving ordering is defined as following: transformation T is defined as ordering preserving in case
* values of the source column A values being ordered as a1, a2, a3 ..., will map into column B = T(A) which
* will keep the same ordering b1, b2, b3, ... with b1 = T(a1), b2 = T(a2), ...
*/
def tryMatchAttributeOrderingPreservingTransformation(expr: Expression): Option[AttributeReference]
private def hasUnresolvedRefs(resolvedExpr: Expression): Boolean =
resolvedExpr.collectFirst {
case _: UnresolvedAttribute | _: UnresolvedFunction => true
}.isDefined
}

View File

@@ -32,16 +32,21 @@ import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row, SparkSession}
import java.util.Locale
/**
* An interface to adapter the difference between spark2 and spark3
* in some spark related class.
* Interface adapting discrepancies and incompatibilities between different Spark versions
*/
trait SparkAdapter extends Serializable {
/**
* Creates instance of [[HoodieCatalystExpressionUtils]] providing for common utils operating
* on Catalyst Expressions
*/
def createCatalystExpressionUtils(): HoodieCatalystExpressionUtils
/**
* Creates instance of [[HoodieAvroSerializer]] providing for ability to serialize
* Spark's [[InternalRow]] into Avro payloads