[HUDI-2102] Support hilbert curve for hudi (#3952)
Co-authored-by: Y Ethan Guo <ethan.guoyihua@gmail.com>
This commit is contained in:
@@ -19,15 +19,16 @@
|
||||
package org.apache.spark.sql.hudi.execution
|
||||
|
||||
import org.apache.hudi.config.HoodieClusteringConfig
|
||||
import org.apache.hudi.optimize.{HilbertCurveUtils, ZOrderingUtil}
|
||||
import org.apache.spark.rdd.{PartitionPruningRDD, RDD}
|
||||
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, BoundReference, SortOrder, UnsafeProjection, UnsafeRow}
|
||||
import org.apache.hudi.optimize.ZOrderingUtil
|
||||
import org.apache.spark.sql.{DataFrame, Row}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering
|
||||
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, BoundReference, SortOrder, UnsafeProjection, UnsafeRow}
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.sql.{DataFrame, Row}
|
||||
import org.apache.spark.util.MutablePair
|
||||
import org.apache.spark.util.random.SamplingUtils
|
||||
import org.davidmoten.hilbert.HilbertCurve
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
@@ -35,10 +36,10 @@ import scala.reflect.{ClassTag, classTag}
|
||||
import scala.util.hashing.byteswap32
|
||||
|
||||
class RangeSample[K: ClassTag, V](
|
||||
zEncodeNum: Int,
|
||||
rdd: RDD[_ <: Product2[K, V]],
|
||||
private var ascend: Boolean = true,
|
||||
val samplePointsPerPartitionHint: Int = 20) extends Serializable {
|
||||
zEncodeNum: Int,
|
||||
rdd: RDD[_ <: Product2[K, V]],
|
||||
private var ascend: Boolean = true,
|
||||
val samplePointsPerPartitionHint: Int = 20) extends Serializable {
|
||||
|
||||
// We allow zEncodeNum = 0, which happens when sorting an empty RDD under the default settings.
|
||||
require(zEncodeNum >= 0, s"Number of zEncodeNum cannot be negative but found $zEncodeNum.")
|
||||
@@ -335,16 +336,21 @@ object RangeSampleSort {
|
||||
}
|
||||
|
||||
/**
|
||||
* create z-order DataFrame by sample
|
||||
* first, sample origin data to get z-cols bounds, then create z-order DataFrame
|
||||
* create optimize DataFrame by sample
|
||||
* first, sample origin data to get order-cols bounds, then apply sort to produce DataFrame
|
||||
* support all type data.
|
||||
* this method need more resource and cost more time than createZIndexedDataFrameByMapValue
|
||||
* this method need more resource and cost more time than createOptimizedDataFrameByMapValue
|
||||
*/
|
||||
def sortDataFrameBySample(df: DataFrame, zCols: Seq[String], fileNum: Int): DataFrame = {
|
||||
def sortDataFrameBySample(df: DataFrame, zCols: Seq[String], fileNum: Int, sortMode: String): DataFrame = {
|
||||
val spark = df.sparkSession
|
||||
val columnsMap = df.schema.fields.map(item => (item.name, item)).toMap
|
||||
val fieldNum = df.schema.fields.length
|
||||
val checkCols = zCols.filter(col => columnsMap(col) != null)
|
||||
val useHilbert = sortMode match {
|
||||
case "hilbert" => true
|
||||
case "z-order" => false
|
||||
case other => throw new IllegalArgumentException(s"new only support z-order/hilbert optimize but find: ${other}")
|
||||
}
|
||||
|
||||
if (zCols.isEmpty || checkCols.isEmpty) {
|
||||
df
|
||||
@@ -366,7 +372,7 @@ object RangeSampleSort {
|
||||
}.filter(_._1 != -1)
|
||||
// Complex type found, use createZIndexedDataFrameByRange
|
||||
if (zFields.length != zCols.length) {
|
||||
return sortDataFrameBySampleSupportAllTypes(df, zCols, fieldNum)
|
||||
return sortDataFrameBySampleSupportAllTypes(df, zCols, fileNum)
|
||||
}
|
||||
|
||||
val rawRdd = df.rdd
|
||||
@@ -441,6 +447,7 @@ object RangeSampleSort {
|
||||
val boundBroadCast = spark.sparkContext.broadcast(expandSampleBoundsWithFactor)
|
||||
|
||||
val indexRdd = rawRdd.mapPartitions { iter =>
|
||||
val hilbertCurve = if (useHilbert) Some(HilbertCurve.bits(32).dimensions(zFields.length)) else None
|
||||
val expandBoundsWithFactor = boundBroadCast.value
|
||||
val maxBoundNum = expandBoundsWithFactor.map(_._1.length).max
|
||||
val longDecisionBound = new RawDecisionBound(Ordering[Long])
|
||||
@@ -507,17 +514,21 @@ object RangeSampleSort {
|
||||
case _ =>
|
||||
-1
|
||||
}
|
||||
}.filter(v => v != -1).map(ZOrderingUtil.intTo8Byte(_)).toArray
|
||||
val zValues = ZOrderingUtil.interleaving(values, 8)
|
||||
Row.fromSeq(row.toSeq ++ Seq(zValues))
|
||||
}.filter(v => v != -1)
|
||||
val mapValues = if (hilbertCurve.isDefined) {
|
||||
HilbertCurveUtils.indexBytes(hilbertCurve.get, values.map(_.toLong).toArray, 32)
|
||||
} else {
|
||||
ZOrderingUtil.interleaving(values.map(ZOrderingUtil.intTo8Byte(_)).toArray, 8)
|
||||
}
|
||||
Row.fromSeq(row.toSeq ++ Seq(mapValues))
|
||||
}
|
||||
}.sortBy(x => ZorderingBinarySort(x.getAs[Array[Byte]](fieldNum)), numPartitions = fileNum)
|
||||
val newDF = df.sparkSession.createDataFrame(indexRdd, StructType(
|
||||
df.schema.fields ++ Seq(
|
||||
StructField(s"zindex",
|
||||
StructField(s"index",
|
||||
BinaryType, false))
|
||||
))
|
||||
newDF.drop("zindex")
|
||||
newDF.drop("index")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user