[
https://issues.apache.org/jira/browse/HUDI-2101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17386632#comment-17386632
]
ASF GitHub Bot commented on HUDI-2101:
--------------------------------------
leesf commented on a change in pull request #3330:
URL: https://github.com/apache/hudi/pull/3330#discussion_r675936746
##########
File path:
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/Zoptimize.scala
##########
@@ -0,0 +1,750 @@
+/*
+ * 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 java.sql.Date
+import java.util.concurrent.{Executors, ThreadPoolExecutor}
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.parquet.hadoop.ParquetFileReader
+
+import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Ascending,
Attribute, AttributeReference, BoundReference, EqualNullSafe, EqualTo,
Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In,
IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, SortOrder,
StartsWith, UnsafeProjection}
+import
org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.hudi.ZOrderingUtil
+import org.apache.spark.sql.hudi.execution._
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.{MutablePair, SerializableConfiguration}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+import scala.concurrent.{ExecutionContext, Future}
+
+object Zoptimize {
+
+ case class FileStats(val minVal: String, val maxVal: String, val num_nulls:
Int = 0)
+ case class ColumnFileStats(val fileName: String, val colName: String, val
minVal: String, val maxVal: String, val num_nulls: Int = 0)
+
+ def createZIndexedDataFrameByRange(df: DataFrame, zCols: String, fileNum:
Int): DataFrame = {
+ createZIndexedDataFrameByRange(df, zCols.split(",").map(_.trim), fileNum)
+ }
+
+ def createZIndexDataFrameBySample(df: DataFrame, zCols: String, fileNum:
Int): DataFrame = {
+ createZIndexDataFrameBySample(df, zCols.split(",").map(_.trim), fileNum)
+ }
+
+ /**
+ * create z-order DataFrame by sample
+ * first, sample origin data to get z-cols bounds, then create z-order
DataFrame
+ * support all type data.
+ * this method need more resource and cost more time than
createZIndexedDataFrameByMapValue
+ */
+ def createZIndexDataFrameBySample(df: DataFrame, zCols: Seq[String],
fileNum: Int): 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)
+
+ if (zCols.isEmpty || checkCols.isEmpty) {
+ df
+ } else {
+ val zFields = zCols.map { col =>
+ val newCol = columnsMap(col)
+ if (newCol == null) {
+ (-1, null)
+ } else {
+ newCol.dataType match {
+ case LongType | DoubleType | FloatType | StringType | IntegerType
| DateType | TimestampType | ShortType | ByteType =>
+ (df.schema.fields.indexOf(newCol), newCol)
+ case d: DecimalType =>
+ (df.schema.fields.indexOf(newCol), newCol)
+ case _ =>
+ (-1, null)
+ }
+ }
+ }.filter(_._1 != -1)
+ // Complex type found, use createZIndexedDataFrameByRange
+ if (zFields.length != zCols.length) {
+ return createZIndexedDataFrameByRange(df, zCols, fieldNum)
+ }
+
+ val rawRdd = df.rdd
+ val sampleRdd = rawRdd.map { row =>
+ val values = zFields.map { case (index, field) =>
+ field.dataType match {
+ case LongType =>
+ if (row.isNullAt(index)) Long.MaxValue else row.getLong(index)
+ case DoubleType =>
+ if (row.isNullAt(index)) Long.MaxValue else
java.lang.Double.doubleToLongBits(row.getDouble(index))
+ case IntegerType =>
+ if (row.isNullAt(index)) Long.MaxValue else
row.getInt(index).toLong
+ case FloatType =>
+ if (row.isNullAt(index)) Long.MaxValue else
java.lang.Double.doubleToLongBits(row.getFloat(index).toDouble)
+ case StringType =>
+ if (row.isNullAt(index)) "" else row.getString(index)
+ case DateType =>
+ if (row.isNullAt(index)) Long.MaxValue else
row.getDate(index).getTime
+ case TimestampType =>
+ if (row.isNullAt(index)) Long.MaxValue else
row.getTimestamp(index).getTime
+ case ByteType =>
+ if (row.isNullAt(index)) Long.MaxValue else
row.getByte(index).toLong
+ case ShortType =>
+ if (row.isNullAt(index)) Long.MaxValue else
row.getShort(index).toLong
+ case d: DecimalType =>
+ if (row.isNullAt(index)) Long.MaxValue else
row.getDecimal(index).longValue()
+ case _ =>
+ null
+ }
+ }.filter(v => v != null).toArray
+ (values, null)
+ }
+ val zOrderBounds =
df.sparkSession.sessionState.conf.getConfString("spark.zorder.bounds.number",
"200000").toInt
+ val sample = new RangeSample(zOrderBounds, sampleRdd)
+ val rangeBounds = sample.getRangeBounds()
+ val sampleBounds = {
+ val candidateColNumber = rangeBounds.head._1.length
+ (0 to candidateColNumber - 1).map { i =>
+ val colRangeBound = rangeBounds.map(x => (x._1(i), x._2))
+
+ if (colRangeBound.head._1.isInstanceOf[String]) {
+
sample.determineBound(colRangeBound.asInstanceOf[ArrayBuffer[(String, Float)]],
math.min(zOrderBounds, rangeBounds.length), Ordering[String])
+ } else {
+
sample.determineBound(colRangeBound.asInstanceOf[ArrayBuffer[(Long, Float)]],
math.min(zOrderBounds, rangeBounds.length), Ordering[Long])
+ }
+ }
+ }
+
+ // expand bounds.
+ // maybe it's better to use the value of "spark.zorder.bounds.number" as
maxLength,
+ // however this will lead to extra time costs when all zorder cols
distinct count values are less then "spark.zorder.bounds.number"
+ val maxLength = sampleBounds.map(_.length).max
+ val expandSampleBoundsWithFactor = sampleBounds.map { bound =>
+ val fillFactor = maxLength / bound.size
+ val newBound = new Array[Double](bound.length * fillFactor)
+ if (bound.isInstanceOf[Array[Long]] && fillFactor > 1) {
+ val longBound = bound.asInstanceOf[Array[Long]]
+ for (i <- 0 to bound.length - 1) {
+ for (j <- 0 to fillFactor - 1) {
+ // sample factor shoud not be too large, so it's ok to use 1 /
fillfactor as slice
+ newBound(j + i*(fillFactor)) = longBound(i) + (j + 1) * (1 /
fillFactor.toDouble)
+ }
+ }
+ (newBound, fillFactor)
+ } else {
+ (bound, 0)
+ }
+ }
+
+ val boundBroadCast =
spark.sparkContext.broadcast(expandSampleBoundsWithFactor)
+
+ val indexRdd = rawRdd.mapPartitions { iter =>
+ val expandBoundsWithFactor = boundBroadCast.value
+ val maxBoundNum = expandBoundsWithFactor.map(_._1.length).max
+ val longDecisionBound = new RawDecisionBound(Ordering[Long])
+ val doubleDecisionBound = new RawDecisionBound(Ordering[Double])
+ val stringDecisionBound = new RawDecisionBound(Ordering[String])
+ import java.util.concurrent.ThreadLocalRandom
+ val threadLocalRandom = ThreadLocalRandom.current
+
+ def getRank(rawIndex: Int, value: Long, isNull: Boolean): Int = {
+ val (expandBound, factor) = expandBoundsWithFactor(rawIndex)
+ if (isNull) {
+ expandBound.length + 1
+ } else {
+ if (factor > 1) {
+ doubleDecisionBound.getBound(value +
(threadLocalRandom.nextInt(factor) + 1)*(1 / factor.toDouble),
expandBound.asInstanceOf[Array[Double]])
+ } else {
+ longDecisionBound.getBound(value,
expandBound.asInstanceOf[Array[Long]])
+ }
+ }
+ }
+
+ iter.map { row =>
+ val values = zFields.zipWithIndex.map { case ((index, field),
rawIndex) =>
+ field.dataType match {
+ case LongType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else row.getLong(index),
isNull)
+ case DoubleType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
java.lang.Double.doubleToLongBits(row.getDouble(index)), isNull)
+ case IntegerType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else row.getInt(index).toLong,
isNull)
+ case FloatType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
java.lang.Double.doubleToLongBits(row.getFloat(index).toDouble), isNull)
+ case StringType =>
+ val factor = maxBoundNum.toDouble /
expandBoundsWithFactor(rawIndex)._1.length
+ if (row.isNullAt(index)) {
+ maxBoundNum + 1
+ } else {
+ val currentRank =
stringDecisionBound.getBound(row.getString(index),
expandBoundsWithFactor(rawIndex)._1.asInstanceOf[Array[String]])
+ if (factor > 1) {
+ (currentRank*factor).toInt +
threadLocalRandom.nextInt(factor.toInt)
+ } else {
+ currentRank
+ }
+ }
+ case DateType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
row.getDate(index).getTime, isNull)
+ case TimestampType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
row.getTimestamp(index).getTime, isNull)
+ case ByteType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
row.getByte(index).toLong, isNull)
+ case ShortType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
row.getShort(index).toLong, isNull)
+ case d: DecimalType =>
+ val isNull = row.isNullAt(index)
+ getRank(rawIndex, if (isNull) 0 else
row.getDecimal(index).longValue(), isNull)
+ case _ =>
+ -1
+ }
+ }.filter(v => v != -1).map(ZOrderingUtil.intTo8Byte(_)).toArray
+ val zValues = ZOrderingUtil.interleaving(values, 8)
+ Row.fromSeq(row.toSeq ++ Seq(zValues))
+ }
+ }.sortBy(x => ZorderingBinarySort(x.getAs[Array[Byte]](fieldNum)),
numPartitions = fileNum)
+ val newDF = df.sparkSession.createDataFrame(indexRdd, StructType(
+ df.schema.fields ++ Seq(
+ StructField(s"zindex",
+ BinaryType, false))
+ ))
+ newDF.drop("zindex")
+ }
+ }
+
+ /**
+ * create z-order DataFrame by sample
+ * support all col types
+ */
+ def createZIndexedDataFrameByRange(df: DataFrame, zCols: Seq[String],
fileNum: Int): DataFrame = {
+ val spark = df.sparkSession
+ val internalRdd = df.queryExecution.toRdd
+ val schema = df.schema
+ val outputAttributes = df.queryExecution.analyzed.output
+ val sortingExpressions = outputAttributes.filter(p =>
zCols.contains(p.name))
+ if (sortingExpressions.length == 0 || sortingExpressions.length !=
zCols.size) {
+ df
+ } else {
+ val zOrderBounds =
spark.sessionState.conf.getConfString("spark.zorder.bounds.number",
"200000").toInt
Review comment:
ditto
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
> support z-order for hudi
> ------------------------
>
> Key: HUDI-2101
> URL: https://issues.apache.org/jira/browse/HUDI-2101
> Project: Apache Hudi
> Issue Type: Sub-task
> Components: Spark Integration
> Reporter: tao meng
> Assignee: tao meng
> Priority: Major
> Labels: pull-request-available
> Fix For: 0.10.0
>
>
> support z-order for hudi to optimze the query
--
This message was sent by Atlassian Jira
(v8.3.4#803005)