You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "LuciferYang (via GitHub)" <gi...@apache.org> on 2023/03/02 16:13:05 UTC

[GitHub] [spark] LuciferYang opened a new pull request, #40255: [SPARK-42558][CONNECT] DataFrameStatFunctions WIP

LuciferYang opened a new pull request, #40255:
URL: https://github.com/apache/spark/pull/40255

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124699930


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),

Review Comment:
   I keep it as it is now



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124797844


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    sparkSession.newDataFrame { builder =>

Review Comment:
   The results of local test and GA test are different
   
   



-- 
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


[GitHub] [spark] hvanhovell commented on pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on PR #40255:
URL: https://github.com/apache/spark/pull/40255#issuecomment-1453850392

   @LuciferYang can you update the binary compatibility tests?


-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125221876


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -133,6 +134,10 @@ object CheckConnectJvmClientCompatibility {
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.jdbc"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameWriter.jdbc"),
 
+      // DataFrameStatFunctions
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.bloomFilter"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.this"),

Review Comment:
   yes you are right.... :)



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124806858


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    sparkSession.newDataFrame { builder =>

Review Comment:
   I changed a simple case to try again
   
   



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124451582


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>

Review Comment:
   `newDataset(PrimitiveDoubleEncoder)`?



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124686005


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),

Review Comment:
   There need `Array[Array[Double]]` Encoder,  need make `newArrayEncoder` as private[sql] and write like 
   ```
   newArrayEncoder(newDoubleArrayEncoder.asInstanceOf[ArrayEncoder[Double]])
   ```
   ?



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124585799


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {

Review Comment:
   @zhengruifeng @amaliujia do you know why we implemented a separate proto messages for this? It seems all of this can be implemented using aggregates.



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125473488


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(

Review Comment:
   it is only py4j? if so, then let's get rid of it.



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124795511


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    sparkSession.newDataFrame { builder =>

Review Comment:
   I don't see a problem. Is it not working?



-- 
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


[GitHub] [spark] LuciferYang commented on pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40255:
URL: https://github.com/apache/spark/pull/40255#issuecomment-1455324716

   Thanks @hvanhovell @HyukjinKwon @zhengruifeng @amaliujia 


-- 
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


[GitHub] [spark] hvanhovell closed pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell closed pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions
URL: https://github.com/apache/spark/pull/40255


-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124696192


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession

Review Comment:
   done



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] DataFrameStatFunctions WIP

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1123365858


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")

Review Comment:
   Missing proto definition, will be implemented in followup
   
    



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, numBits, Double.NaN)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, numBits, Double.NaN)
+  }
+
+  private def buildBloomFilter(
+      col: Column,
+      expectedNumItems: Long,
+      numBits: Long,
+      fpp: Double): BloomFilter = {
+    throw new UnsupportedOperationException("buildBloomFilter is not implemented.")

Review Comment:
   Missing proto definition, will be implemented in followup
   



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125217576


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -133,6 +134,10 @@ object CheckConnectJvmClientCompatibility {
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.jdbc"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameWriter.jdbc"),
 
+      // DataFrameStatFunctions
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.bloomFilter"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.this"),

Review Comment:
   Please also remove `ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.na")`



-- 
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


[GitHub] [spark] LuciferYang commented on pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40255:
URL: https://github.com/apache/spark/pull/40255#issuecomment-1454330973

   Now all paased


-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124736465


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")

Review Comment:
   Thanks ~



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125222488


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -133,6 +134,10 @@ object CheckConnectJvmClientCompatibility {
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.jdbc"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameWriter.jdbc"),
 
+      // DataFrameStatFunctions
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.bloomFilter"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.this"),

Review Comment:
   done



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124462028


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, numBits, Double.NaN)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, numBits, Double.NaN)
+  }
+
+  private def buildBloomFilter(
+      col: Column,
+      expectedNumItems: Long,
+      numBits: Long,
+      fpp: Double): BloomFilter = {
+    throw new UnsupportedOperationException("buildBloomFilter is not implemented.")

Review Comment:
   Please remove this for now. Can you file an follow-up ticket.



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124461534


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")

Review Comment:
   You can implement this by using the aggregate function:
   ```scala
   val agg = Column.fn("count_min_sketch", col.expr, lit(eps), lit(confidence), lit(seed))
   val ds = sparkSession.newDataset(BinaryEncoder) { builder =>
     builder.getProjectBuilder
       .setInput(plan.getRoot)
       .addExpressions(agg.expr)
   }
   CountMinSketch.readFrom(ds.head())
   ```



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125221046


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -133,6 +134,10 @@ object CheckConnectJvmClientCompatibility {
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameReader.jdbc"),
       ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameWriter.jdbc"),
 
+      // DataFrameStatFunctions
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.bloomFilter"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameStatFunctions.this"),

Review Comment:
   Are you want to  say `org.apache.spark.sql.Dataset.stat`?  `na` is `DataFrameNaFunctions`, not `DataFrameStatFunctions`



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124580063


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, numBits, Double.NaN)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, numBits, Double.NaN)
+  }
+
+  private def buildBloomFilter(
+      col: Column,
+      expectedNumItems: Long,
+      numBits: Long,
+      fpp: Double): BloomFilter = {
+    throw new UnsupportedOperationException("buildBloomFilter is not implemented.")

Review Comment:
   we can also add the aggregate function, but I would do that in a separate pr



-- 
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


[GitHub] [spark] amaliujia commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "amaliujia (via GitHub)" <gi...@apache.org>.
amaliujia commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124913199


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {

Review Comment:
   I don't remember the decision now. Will need @zhengruifeng to comment



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124788183


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataset(PrimitiveDoubleEncoder) { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    sparkSession.newDataFrame { builder =>

Review Comment:
   @hvanhovell Is there any error in the construction of this dataframe?
   
   



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124775799


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),

Review Comment:
   This is fine. Can you put it in a separate val? that will make the code more readable. You can also put it in static since the encoder is immutable.



-- 
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


[GitHub] [spark] zhengruifeng commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "zhengruifeng (via GitHub)" <gi...@apache.org>.
zhengruifeng commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125375617


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(

Review Comment:
   do we need this function?
   
   python client will not invoke this method in scala client via py4j.
   
   https://github.com/apache/spark/blob/3d900b70c5593326ddc96f094d9abe796308b0e4/python/pyspark/sql/dataframe.py#L4433



-- 
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


[GitHub] [spark] hvanhovell commented on pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on PR #40255:
URL: https://github.com/apache/spark/pull/40255#issuecomment-1455323028

   Merging.


-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124450308


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),

Review Comment:
   You can use `sparkSession.implicits.newDoubleArrayEncoder` here for the nested encoder.



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] DataFrameStatFunctions WIP

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1123361868


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala:
##########
@@ -586,6 +586,57 @@ class ClientE2ETestSuite extends RemoteSparkSession {
       list.asScala.map(kv => Row(kv.key, kv.value)),
       session.createDataFrame(list.asScala.toSeq))
   }
+
+  test("approximate quantile") {

Review Comment:
   will add more case later



-- 
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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124451930


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession

Review Comment:
   newDataset(PrimitiveDoubleEncoder)?



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession

Review Comment:
   `newDataset(PrimitiveDoubleEncoder)`?



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124686005


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,670 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),

Review Comment:
   There need `Array[Array[Double]]` Encoder,  need make `sparkSession.implicits.newArrayEncoder` as private[sql] and write like 
   ```
   newArrayEncoder(newDoubleArrayEncoder.asInstanceOf[ArrayEncoder[Double]])
   ```
   ?



-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1124699079


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,665 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.Relation
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch}
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0,
+      s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(
+        ArrayEncoder(
+          ArrayEncoder(PrimitiveDoubleEncoder, containsNull = false),
+          containsNull = false)) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(
+      cols: List[String],
+      probabilities: List[Double],
+      relativeError: Double): java.util.List[java.util.List[Double]] = {
+    approxQuantile(cols.toArray, probabilities.toArray, relativeError)
+      .map(_.toList.asJava)
+      .toList
+      .asJava
+  }
+
+  /**
+   * Calculate the sample covariance of two numerical columns of a DataFrame.
+   * @param col1
+   *   the name of the first column
+   * @param col2
+   *   the name of the second column
+   * @return
+   *   the covariance of the two columns.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.cov("rand1", "rand2")
+   *    res1: Double = 0.065...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def cov(col1: String, col2: String): Double = {
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCovBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson
+   * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in
+   * MLlib's Statistics.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String, method: String): Double = {
+    require(
+      method == "pearson",
+      "Currently only the calculation of the Pearson Correlation " +
+        "coefficient is supported.")
+    sparkSession
+      .newDataFrame { builder =>
+        builder.getCorrBuilder.setInput(root).setCol1(col1).setCol2(col2)
+      }
+      .head()
+      .getDouble(0)
+  }
+
+  /**
+   * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame.
+   *
+   * @param col1
+   *   the name of the column
+   * @param col2
+   *   the name of the column to calculate the correlation against
+   * @return
+   *   The Pearson Correlation Coefficient as a Double.
+   *
+   * {{{
+   *    val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10))
+   *      .withColumn("rand2", rand(seed=27))
+   *    df.stat.corr("rand1", "rand2", "pearson")
+   *    res1: Double = 0.613...
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def corr(col1: String, col2: String): Double = {
+    corr(col1, col2, "pearson")
+  }
+
+  /**
+   * Computes a pair-wise frequency table of the given columns. Also known as a contingency table.
+   * The first column of each row will be the distinct values of `col1` and the column names will
+   * be the distinct values of `col2`. The name of the first column will be `col1_col2`. Counts
+   * will be returned as `Long`s. Pairs that have no occurrences will have zero as their counts.
+   * Null elements will be replaced by "null", and back ticks will be dropped from elements if
+   * they exist.
+   *
+   * @param col1
+   *   The name of the first column. Distinct items will make the first item of each row.
+   * @param col2
+   *   The name of the second column. Distinct items will make the column names of the DataFrame.
+   * @return
+   *   A DataFrame containing for the contingency table.
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3)))
+   *      .toDF("key", "value")
+   *    val ct = df.stat.crosstab("key", "value")
+   *    ct.show()
+   *    +---------+---+---+---+
+   *    |key_value|  1|  2|  3|
+   *    +---------+---+---+---+
+   *    |        2|  2|  0|  1|
+   *    |        1|  1|  1|  0|
+   *    |        3|  0|  1|  1|
+   *    +---------+---+---+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def crosstab(col1: String, col2: String): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      builder.getCrosstabBuilder.setInput(root).setCol1(col1).setCol2(col2)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. The `support` should be greater than 1e-4.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @param support
+   *   The minimum frequency for an item to be considered `frequent`. Should be greater than 1e-4.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String], support: Double): DataFrame = {
+    sparkSession.newDataFrame { builder =>
+      val freqItemsBuilder = builder.getFreqItemsBuilder.setInput(root).setSupport(support)
+      cols.foreach(freqItemsBuilder.addCols)
+    }
+  }
+
+  /**
+   * Finding frequent items for columns, possibly with false positives. Using the frequent element
+   * count algorithm described in <a href="https://doi.org/10.1145/762471.762473">here</a>,
+   * proposed by Karp, Schenker, and Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Array[String]): DataFrame = {
+    freqItems(cols, 0.01)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * {{{
+   *    val rows = Seq.tabulate(100) { i =>
+   *      if (i % 2 == 0) (1, -1.0) else (i, i * -1.0)
+   *    }
+   *    val df = spark.createDataFrame(rows).toDF("a", "b")
+   *    // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns
+   *    // "a" and "b"
+   *    val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4)
+   *    freqSingles.show()
+   *    +-----------+-------------+
+   *    |a_freqItems|  b_freqItems|
+   *    +-----------+-------------+
+   *    |    [1, 99]|[-1.0, -99.0]|
+   *    +-----------+-------------+
+   *    // find the pair of items with a frequency greater than 0.1 in columns "a" and "b"
+   *    val pairDf = df.select(struct("a", "b").as("a-b"))
+   *    val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1)
+   *    freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show()
+   *    +----------+
+   *    |   freq_ab|
+   *    +----------+
+   *    |  [1,-1.0]|
+   *    |   ...    |
+   *    +----------+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String], support: Double): DataFrame = {
+    freqItems(cols.toArray, support)
+  }
+
+  /**
+   * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the
+   * frequent element count algorithm described in <a
+   * href="https://doi.org/10.1145/762471.762473">here</a>, proposed by Karp, Schenker, and
+   * Papadimitriou. Uses a `default` support of 1%.
+   *
+   * This function is meant for exploratory data analysis, as we make no guarantee about the
+   * backward compatibility of the schema of the resulting `DataFrame`.
+   *
+   * @param cols
+   *   the names of the columns to search frequent items in.
+   * @return
+   *   A Local DataFrame with the Array of frequent items for each column.
+   *
+   * @since 3.4.0
+   */
+  def freqItems(cols: Seq[String]): DataFrame = {
+    freqItems(cols.toArray, 0.01)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * {{{
+   *    val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2),
+   *      (3, 3))).toDF("key", "value")
+   *    val fractions = Map(1 -> 1.0, 3 -> 0.5)
+   *    df.stat.sampleBy("key", fractions, 36L).show()
+   *    +---+-----+
+   *    |key|value|
+   *    +---+-----+
+   *    |  1|    1|
+   *    |  1|    2|
+   *    |  3|    2|
+   *    +---+-----+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = {
+    sampleBy(Column(col), fractions, seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: String, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Returns a stratified sample without replacement based on the fraction given on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * The stratified sample can be performed over multiple columns:
+   * {{{
+   *    import org.apache.spark.sql.Row
+   *    import org.apache.spark.sql.functions.struct
+   *
+   *    val df = spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
+   *      ("Alice", 10))).toDF("name", "age")
+   *    val fractions = Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0)
+   *    df.stat.sampleBy(struct($"name", $"age"), fractions, 36L).show()
+   *    +-----+---+
+   *    | name|age|
+   *    +-----+---+
+   *    | Nico|  8|
+   *    |Alice| 10|
+   *    +-----+---+
+   * }}}
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: Map[T, Double], seed: Long): DataFrame = {
+    require(
+      fractions.values.forall(p => p >= 0.0 && p <= 1.0),
+      s"Fractions must be in [0, 1], but got $fractions.")
+    import org.apache.spark.sql.functions.{rand, udf}
+    val r = rand(seed)
+    val f = udf { (stratum: Any, x: Double) =>
+      x < fractions.getOrElse(stratum.asInstanceOf[T], 0.0)
+    }
+    sparkSession.newDataFrame { builder =>
+      builder.getFilterBuilder.setInput(root).setCondition(f(col, r).expr)
+    }
+  }
+
+  /**
+   * (Java-specific) Returns a stratified sample without replacement based on the fraction given
+   * on each stratum.
+   * @param col
+   *   column that defines strata
+   * @param fractions
+   *   sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as
+   *   zero.
+   * @param seed
+   *   random seed
+   * @tparam T
+   *   stratum type
+   * @return
+   *   a new `DataFrame` that represents the stratified sample
+   *
+   * @since 3.4.0
+   */
+  def sampleBy[T](col: Column, fractions: ju.Map[T, jl.Double], seed: Long): DataFrame = {
+    sampleBy(col, fractions.asScala.toMap.asInstanceOf[Map[T, Double]], seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(colName: String, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), depth, width, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(
+      colName: String,
+      eps: Double,
+      confidence: Double,
+      seed: Int): CountMinSketch = {
+    countMinSketch(Column(colName), eps, confidence, seed)
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param depth
+   *   depth of the sketch
+   * @param width
+   *   width of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, depth: Int, width: Int, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(depth, width, seed))
+  }
+
+  /**
+   * Builds a Count-min Sketch over a specified column.
+   *
+   * @param col
+   *   the column over which the sketch is built
+   * @param eps
+   *   relative error of the sketch
+   * @param confidence
+   *   confidence of the sketch
+   * @param seed
+   *   random seed
+   * @return
+   *   a `CountMinSketch` over column `colName`
+   * @since 3.4.0
+   */
+  def countMinSketch(col: Column, eps: Double, confidence: Double, seed: Int): CountMinSketch = {
+    countMinSketch(col, CountMinSketch.create(eps, confidence, seed))
+  }
+
+  private def countMinSketch(col: Column, zero: CountMinSketch): CountMinSketch = {
+    throw new UnsupportedOperationException("countMinSketch is not implemented.")
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param fpp
+   *   expected false positive probability of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, fpp: Double): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, -1L, fpp)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param colName
+   *   name of the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(colName: String, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(Column(colName), expectedNumItems, numBits, Double.NaN)
+  }
+
+  /**
+   * Builds a Bloom filter over a specified column.
+   *
+   * @param col
+   *   the column over which the filter is built
+   * @param expectedNumItems
+   *   expected number of items which will be put into the filter.
+   * @param numBits
+   *   expected number of bits of the filter.
+   * @since 3.4.0
+   */
+  def bloomFilter(col: Column, expectedNumItems: Long, numBits: Long): BloomFilter = {
+    buildBloomFilter(col, expectedNumItems, numBits, Double.NaN)
+  }
+
+  private def buildBloomFilter(
+      col: Column,
+      expectedNumItems: Long,
+      numBits: Long,
+      fpp: Double): BloomFilter = {
+    throw new UnsupportedOperationException("buildBloomFilter is not implemented.")

Review Comment:
   create [SPARK-42664](https://issues.apache.org/jira/browse/SPARK-42664)



-- 
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


[GitHub] [spark] LuciferYang commented on pull request #40255: [SPARK-42558][CONNECT] Partial implement `DataFrameStatFunctions`

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40255:
URL: https://github.com/apache/spark/pull/40255#issuecomment-1453872602

   > @LuciferYang can you update the binary compatibility tests?
   
   done


-- 
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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #40255:
URL: https://github.com/apache/spark/pull/40255#discussion_r1125591758


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala:
##########
@@ -0,0 +1,605 @@
+/*
+ * 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.{lang => jl, util => ju}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto.{Relation, StatSampleBy}
+import org.apache.spark.sql.DataFrameStatFunctions.approxQuantileResultEncoder
+import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ArrayEncoder, BinaryEncoder, PrimitiveDoubleEncoder}
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.util.sketch.CountMinSketch
+
+/**
+ * Statistic functions for `DataFrame`s.
+ *
+ * @since 3.4.0
+ */
+final class DataFrameStatFunctions private[sql] (sparkSession: SparkSession, root: Relation) {
+
+  /**
+   * Calculates the approximate quantiles of a numerical column of a DataFrame.
+   *
+   * The result of this algorithm has the following deterministic bound: If the DataFrame has N
+   * elements and if we request the quantile at probability `p` up to error `err`, then the
+   * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is
+   * close to (p * N). More precisely,
+   *
+   * {{{
+   *   floor((p - err) * N) <= rank(x) <= ceil((p + err) * N)
+   * }}}
+   *
+   * This method implements a variation of the Greenwald-Khanna algorithm (with some speed
+   * optimizations). The algorithm was first present in <a
+   * href="https://doi.org/10.1145/375663.375670"> Space-efficient Online Computation of Quantile
+   * Summaries</a> by Greenwald and Khanna.
+   *
+   * @param col
+   *   the name of the numerical column
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities
+   *
+   * @note
+   *   null and NaN values will be removed from the numerical column before calculation. If the
+   *   dataframe is empty or the column only contains null or NaN, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      col: String,
+      probabilities: Array[Double],
+      relativeError: Double): Array[Double] = {
+    approxQuantile(Array(col), probabilities, relativeError).head
+  }
+
+  /**
+   * Calculates the approximate quantiles of numerical columns of a DataFrame.
+   * @see
+   *   `approxQuantile(col:Str* approxQuantile)` for detailed description.
+   *
+   * @param cols
+   *   the names of the numerical columns
+   * @param probabilities
+   *   a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the
+   *   minimum, 0.5 is the median, 1 is the maximum.
+   * @param relativeError
+   *   The relative target precision to achieve (greater than or equal to 0). If set to zero, the
+   *   exact quantiles are computed, which could be very expensive. Note that values greater than
+   *   1 are accepted but give the same result as 1.
+   * @return
+   *   the approximate quantiles at the given probabilities of each column
+   *
+   * @note
+   *   null and NaN values will be ignored in numerical columns before calculation. For columns
+   *   only containing null or NaN values, an empty array is returned.
+   *
+   * @since 3.4.0
+   */
+  def approxQuantile(
+      cols: Array[String],
+      probabilities: Array[Double],
+      relativeError: Double): Array[Array[Double]] = {
+    require(
+      probabilities.forall(p => p >= 0.0 && p <= 1.0),
+      "percentile should be in the range [0.0, 1.0]")
+    require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError")
+    sparkSession
+      .newDataset(approxQuantileResultEncoder) { builder =>
+        val approxQuantileBuilder = builder.getApproxQuantileBuilder
+          .setInput(root)
+          .setRelativeError(relativeError)
+        cols.foreach(approxQuantileBuilder.addCols)
+        probabilities.foreach(approxQuantileBuilder.addProbabilities)
+      }
+      .head()
+  }
+
+  /**
+   * Python-friendly version of [[approxQuantile()]]
+   */
+  private[spark] def approxQuantile(

Review Comment:
   already removed



-- 
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


[GitHub] [spark] LuciferYang commented on pull request #40255: [SPARK-42558][CONNECT] Implement `DataFrameStatFunctions` except `bloomFilter` functions

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #40255:
URL: https://github.com/apache/spark/pull/40255#issuecomment-1454994633

   GA passed


-- 
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