You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/10/25 05:03:26 UTC

[GitHub] [spark] cloud-fan commented on a diff in pull request #38375: [SPARK-40900][SQL] Reimplement `frequentItems` with dataframe operations

cloud-fan commented on code in PR #38375:
URL: https://github.com/apache/spark/pull/38375#discussion_r1004000396


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala:
##########
@@ -85,42 +51,144 @@ object FrequentItems extends Logging {
       cols: Seq[String],
       support: Double): DataFrame = {
     require(support >= 1e-4 && support <= 1.0, s"Support must be in [1e-4, 1], but got $support.")
-    val numCols = cols.length
+
     // number of max items to keep counts for
     val sizeOfMap = (1 / support).toInt
-    val countMaps = Seq.tabulate(numCols)(i => new FreqItemCounter(sizeOfMap))
-
-    val freqItems = df.select(cols.map(Column(_)) : _*).rdd.treeAggregate(countMaps)(
-      seqOp = (counts, row) => {
-        var i = 0
-        while (i < numCols) {
-          val thisMap = counts(i)
-          val key = row.get(i)
-          thisMap.add(key, 1L)
-          i += 1
-        }
-        counts
-      },
-      combOp = (baseCounts, counts) => {
-        var i = 0
-        while (i < numCols) {
-          baseCounts(i).merge(counts(i))
-          i += 1
+
+    val frequentItemCols = cols.map { col =>
+      val aggExpr = new CollectFrequentItems(functions.col(col).expr, sizeOfMap)
+      Column(aggExpr.toAggregateExpression(isDistinct = false)).as(s"${col}_freqItems")
+    }
+
+    df.select(frequentItemCols: _*)
+  }
+}
+
+case class CollectFrequentItems(
+    child: Expression,
+    size: Int,
+    mutableAggBufferOffset: Int = 0,
+    inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[mutable.Map[Any, Long]]
+  with ImplicitCastInputTypes with UnaryLike[Expression] {
+  require(size > 0)
+
+  def this(child: Expression, size: Int) = this(child, size, 0, 0)
+
+  // Returns empty array for empty inputs
+  override def nullable: Boolean = false
+
+  override def dataType: DataType = ArrayType(child.dataType, containsNull = child.nullable)
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)

Review Comment:
   seems we don't have any input type requirement, we don't need to extend `ImplicitCastInputTypes`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org