1
0

[HUDI-4250][HUDI-4202] Optimize performance of Column Stats Index reading in Data Skipping (#5746)

We provide an alternative way of fetching Column Stats Index within the reading process to avoid the penalty of a more heavy-weight execution scheduled through a Spark engine.
This commit is contained in:
Alexey Kudinkin
2022-07-25 15:36:12 -07:00
committed by GitHub
parent 6e7ac45735
commit e7c8df7e8b
20 changed files with 730 additions and 387 deletions

View File

@@ -17,15 +17,43 @@
package org.apache.hudi.util
import org.apache.hudi.common.function.{SerializableFunction, SerializablePairFunction}
import org.apache.hudi.common.util.collection
import scala.language.implicitConversions
/**
* Utility allowing for seamless conversion b/w Java/Scala functional primitives
*/
object JFunction {
def toScala[T, R](f: java.util.function.Function[T, R]): T => R =
////////////////////////////////////////////////////////////
// From Java to Scala
////////////////////////////////////////////////////////////
implicit def toScala[T, R](f: java.util.function.Function[T, R]): T => R =
(t: T) => f.apply(t)
def toJava[T](f: T => Unit): java.util.function.Consumer[T] =
////////////////////////////////////////////////////////////
// From Scala to Java
////////////////////////////////////////////////////////////
implicit def toJavaFunction[T, R](f: Function[T, R]): java.util.function.Function[T, R] =
new java.util.function.Function[T, R] {
override def apply(t: T): R = f.apply(t)
}
implicit def toJavaSerializableFunction[T, R](f: Function[T, R]): SerializableFunction[T, R] =
new SerializableFunction[T, R] {
override def apply(t: T): R = f.apply(t)
}
implicit def toJavaSerializablePairFunction[T, K, V](f: Function[T, collection.Pair[K, V]]): SerializablePairFunction[T, K, V] =
new SerializablePairFunction[T, K, V] {
override def call(t: T): collection.Pair[K, V] = f.apply(t)
}
implicit def toJava[T](f: T => Unit): java.util.function.Consumer[T] =
new java.util.function.Consumer[T] {
override def accept(t: T): Unit = f.apply(t)
}

View File

@@ -27,12 +27,16 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, SubqueryAlias}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
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.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SparkSession}
import org.apache.spark.storage.StorageLevel
import java.util.Locale
@@ -138,4 +142,9 @@ trait SparkAdapter extends Serializable {
* TODO move to HoodieCatalystExpressionUtils
*/
def createInterpretedPredicate(e: Expression): InterpretedPredicate
/**
* Converts instance of [[StorageLevel]] to a corresponding string
*/
def convertStorageLevelToString(level: StorageLevel): String
}

View File

@@ -1504,7 +1504,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// prefix search for column (_hoodie_record_key)
ColumnIndexID columnIndexID = new ColumnIndexID(HoodieRecord.RECORD_KEY_METADATA_FIELD);
List<HoodieRecord<HoodieMetadataPayload>> result = tableMetadata.getRecordsByKeyPrefixes(Collections.singletonList(columnIndexID.asBase64EncodedString()),
MetadataPartitionType.COLUMN_STATS.getPartitionPath()).collectAsList();
MetadataPartitionType.COLUMN_STATS.getPartitionPath(), true).collectAsList();
// there are 3 partitions in total and 2 commits. total entries should be 6.
assertEquals(result.size(), 6);
@@ -1515,7 +1515,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// prefix search for col(_hoodie_record_key) and first partition. only 2 files should be matched
PartitionIndexID partitionIndexID = new PartitionIndexID(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
result = tableMetadata.getRecordsByKeyPrefixes(Collections.singletonList(columnIndexID.asBase64EncodedString().concat(partitionIndexID.asBase64EncodedString())),
MetadataPartitionType.COLUMN_STATS.getPartitionPath()).collectAsList();
MetadataPartitionType.COLUMN_STATS.getPartitionPath(), true).collectAsList();
// 1 partition and 2 commits. total entries should be 2.
assertEquals(result.size(), 2);
result.forEach(entry -> {
@@ -1534,7 +1534,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
// prefix search for column {commit time} and first partition
columnIndexID = new ColumnIndexID(HoodieRecord.COMMIT_TIME_METADATA_FIELD);
result = tableMetadata.getRecordsByKeyPrefixes(Collections.singletonList(columnIndexID.asBase64EncodedString().concat(partitionIndexID.asBase64EncodedString())),
MetadataPartitionType.COLUMN_STATS.getPartitionPath()).collectAsList();
MetadataPartitionType.COLUMN_STATS.getPartitionPath(), true).collectAsList();
// 1 partition and 2 commits. total entries should be 2.
assertEquals(result.size(), 2);