zhengruifeng commented on code in PR #38375:
URL: https://github.com/apache/spark/pull/38375#discussion_r1003976467


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala:
##########
@@ -85,42 +51,142 @@ 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)
+
+  override def prettyName: String = "collect_frequent_items"
+
+  override def createAggregationBuffer(): mutable.Map[Any, Long] =
+    mutable.Map.empty[Any, Long]
+
+  private def add(map: mutable.Map[Any, Long], key: Any, count: Long): 
mutable.Map[Any, Long] = {
+    if (map.contains(key)) {
+      map(key) += count
+    } else {
+      if (map.size < size) {
+        map += key -> count
+      } else {
+        val minCount = if (map.values.isEmpty) 0 else map.values.min
+        val remainder = count - minCount
+        if (remainder >= 0) {
+          map += key -> count // something will get kicked out, so we can add 
this
+          map.retain((k, v) => v > minCount)
+          map.transform((k, v) => v - minCount)
+        } else {
+          map.transform((k, v) => v - count)
         }
-        baseCounts
       }
-    )
-    val justItems = freqItems.map(m => m.baseMap.keys.toArray)
-    val resultRow = Row(justItems : _*)
+    }
+    map
+  }
+
+  override def update(
+      buffer: mutable.Map[Any, Long],
+      input: InternalRow): mutable.Map[Any, Long] = {
+    val key = child.eval(input)
+    if (key != null) {
+      this.add(buffer, InternalRow.copyValue(key), 1L)
+    } else {
+      this.add(buffer, key, 1L)
+    }
+  }
+
+  override def merge(
+      buffer: mutable.Map[Any, Long],
+      input: mutable.Map[Any, Long]): mutable.Map[Any, Long] = {
+    input.foreach { case (k, v) =>
+      add(buffer, k, v)
+    }
+    buffer
+  }
 
-    val outputCols = cols.map { name =>
-      val originalField = df.resolve(name)
+  override def eval(buffer: mutable.Map[Any, Long]): Any =
+    new GenericArrayData(buffer.keys.toArray)
 
-      // append frequent Items to the column name for easy debugging
-      StructField(name + "_freqItems", ArrayType(originalField.dataType, 
originalField.nullable))
-    }.toArray
+  private lazy val projection =
+    UnsafeProjection.create(Array[DataType](child.dataType, LongType))
 
-    val schema = StructType(outputCols).toAttributes
-    Dataset.ofRows(df.sparkSession, LocalRelation.fromExternalRows(schema, 
Seq(resultRow)))
+  override def serialize(map: mutable.Map[Any, Long]): Array[Byte] = {
+    val buffer = new Array[Byte](4 << 10) // 4K
+    val bos = new ByteArrayOutputStream()
+    val out = new DataOutputStream(bos)
+    try {

Review Comment:
   good point, will update



-- 
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-unsubscr...@spark.apache.org

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


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

Reply via email to