Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
100 commits
Select commit Hold shift + click to select a range
86db9b2
[SPARK-22833][IMPROVEMENT] in SparkHive Scala Examples
chetkhatri Dec 23, 2017
ea2642e
[SPARK-20694][EXAMPLES] Update SQLDataSourceExample.scala
CNRui Dec 23, 2017
f6084a8
[HOTFIX] Fix Scala style checks
HyukjinKwon Dec 23, 2017
aeb45df
[SPARK-22844][R] Adds date_trunc in R API
HyukjinKwon Dec 23, 2017
1219d7a
[SPARK-22889][SPARKR] Set overwrite=T when install SparkR in tests
shivaram Dec 23, 2017
0bf1a74
[SPARK-22465][CORE] Add a safety-check to RDD defaultPartitioner
Dec 24, 2017
fba0313
[SPARK-22707][ML] Optimize CrossValidator memory occupation by models…
WeichenXu123 Dec 25, 2017
33ae243
[SPARK-22893][SQL] Unified the data type mismatch message
wangyum Dec 25, 2017
12d20dd
[SPARK-22874][PYSPARK][SQL][FOLLOW-UP] Modify error messages to show …
ueshin Dec 25, 2017
be03d3a
[SPARK-22893][SQL][HOTFIX] Fix a error message of VersionsSuite
dongjoon-hyun Dec 26, 2017
0e68330
[SPARK-20168][DSTREAM] Add changes to use kinesis fetches from specif…
yashs360 Dec 26, 2017
eb386be
[SPARK-21552][SQL] Add DecimalType support to ArrowWriter.
ueshin Dec 26, 2017
ff48b1b
[SPARK-22901][PYTHON] Add deterministic flag to pyspark UDF
mgaido91 Dec 26, 2017
9348e68
[SPARK-22833][EXAMPLE] Improvement SparkHive Scala Examples
cloud-fan Dec 26, 2017
91d1b30
[SPARK-22894][SQL] DateTimeOperations should accept SQL like string type
wangyum Dec 26, 2017
6674acd
[SPARK-22846][SQL] Fix table owner is null when creating table throug…
Dec 27, 2017
b8bfce5
[SPARK-22324][SQL][PYTHON][FOLLOW-UP] Update setup.py file.
ueshin Dec 27, 2017
774715d
[SPARK-22904][SQL] Add tests for decimal operations and string casts
mgaido91 Dec 27, 2017
753793b
[SPARK-22899][ML][STREAMING] Fix OneVsRestModel transform on streamin…
WeichenXu123 Dec 28, 2017
5683984
[SPARK-18016][SQL][FOLLOW-UP] Code Generation: Constant Pool Limit - …
kiszk Dec 28, 2017
32ec269
[SPARK-22909][SS] Move Structured Streaming v2 APIs to streaming folder
zsxwing Dec 28, 2017
171f6dd
[SPARK-22757][KUBERNETES] Enable use of remote dependencies (http, s3…
liyinan926 Dec 28, 2017
ded6d27
[SPARK-22648][K8S] Add documentation covering init containers and sec…
liyinan926 Dec 28, 2017
76e8a1d
[SPARK-22843][R] Adds localCheckpoint in R
HyukjinKwon Dec 28, 2017
1eebfbe
[SPARK-21208][R] Adds setLocalProperty and getLocalProperty in R
HyukjinKwon Dec 28, 2017
755f2f5
[SPARK-20392][SQL][FOLLOWUP] should not add extra AnalysisBarrier
cloud-fan Dec 28, 2017
2877817
[SPARK-22917][SQL] Should not try to generate histogram for empty/nul…
Dec 28, 2017
5536f31
[MINOR][BUILD] Fix Java linter errors
dongjoon-hyun Dec 28, 2017
8f6d573
[SPARK-22875][BUILD] Assembly build fails for a high user id
gerashegalov Dec 28, 2017
9c21ece
[SPARK-22836][UI] Show driver logs in UI when available.
Dec 28, 2017
613b71a
[SPARK-22890][TEST] Basic tests for DateTimeOperations
wangyum Dec 28, 2017
cfcd746
[SPARK-11035][CORE] Add in-process Spark app launcher.
Dec 28, 2017
ffe6fd7
[SPARK-22818][SQL] csv escape of quote escape
Dec 28, 2017
c745730
[SPARK-22905][MLLIB] Fix ChiSqSelectorModel save implementation
WeichenXu123 Dec 29, 2017
796e48c
[SPARK-22313][PYTHON][FOLLOWUP] Explicitly import warnings namespace …
HyukjinKwon Dec 29, 2017
67ea11e
[SPARK-22891][SQL] Make hive client creation thread safe
Dec 29, 2017
d4f0b1d
[SPARK-22834][SQL] Make insertion commands have real children to fix …
gengliangwang Dec 29, 2017
224375c
[SPARK-22892][SQL] Simplify some estimation logic by using double ins…
Dec 29, 2017
cc30ef8
[SPARK-22916][SQL] shouldn't bias towards build right if user does no…
Dec 29, 2017
fcf66a3
[SPARK-21657][SQL] optimize explode quadratic memory consumpation
uzadude Dec 29, 2017
dbd492b
[SPARK-22921][PROJECT-INFRA] Choices for Assigning Jira on Merge
squito Dec 29, 2017
11a849b
[SPARK-22370][SQL][PYSPARK][FOLLOW-UP] Fix a test failure when xmlrun…
ueshin Dec 29, 2017
8b49704
[SPARK-20654][CORE] Add config to limit disk usage of the history ser…
Dec 29, 2017
4e9e6ae
[SPARK-22864][CORE] Disable allocation schedule in ExecutorAllocation…
Dec 29, 2017
afc3641
[SPARK-22905][ML][FOLLOWUP] Fix GaussianMixtureModel save
zhengruifeng Dec 29, 2017
66a7d6b
[SPARK-22920][SPARKR] sql functions for current_date, current_timesta…
felixcheung Dec 29, 2017
ccda75b
[SPARK-22921][PROJECT-INFRA] Bug fix in jira assigning
squito Dec 29, 2017
30fcdc0
[SPARK-22922][ML][PYSPARK] Pyspark portion of the fit-multiple API
MrBago Dec 30, 2017
8169630
[SPARK-22734][ML][PYSPARK] Added Python API for VectorSizeHint.
MrBago Dec 30, 2017
2ea17af
[SPARK-22881][ML][TEST] ML regression package testsuite add Structure…
WeichenXu123 Dec 30, 2017
f2b3525
[SPARK-22771][SQL] Concatenate binary inputs into a binary output
maropu Dec 30, 2017
14c4a62
[SPARK-21475][Core]Revert "[SPARK-21475][CORE] Use NIO's Files API to…
zsxwing Dec 30, 2017
234d943
[TEST][MINOR] remove redundant `EliminateSubqueryAliases` in test code
wzhfy Dec 30, 2017
fd7d141
[SPARK-22919] Bump httpclient versions
Dec 30, 2017
ea0a5ee
[SPARK-22924][SPARKR] R API for sortWithinPartitions
felixcheung Dec 30, 2017
ee3af15
[SPARK-22363][SQL][TEST] Add unit test for Window spilling
gaborgsomogyi Dec 31, 2017
cfbe11e
[SPARK-22895][SQL] Push down the deterministic predicates that are af…
gatorsmile Dec 31, 2017
3d8837e
[SPARK-22397][ML] add multiple columns support to QuantileDiscretizer
huaxingao Dec 31, 2017
028ee40
[SPARK-22801][ML][PYSPARK] Allow FeatureHasher to treat numeric colum…
Dec 31, 2017
5955a2d
[MINOR][DOCS] s/It take/It takes/g
jkremser Dec 31, 2017
994065d
[SPARK-13030][ML] Create OneHotEncoderEstimator for OneHotEncoder as …
viirya Dec 31, 2017
f5b7714
[BUILD] Close stale PRs
srowen Jan 1, 2018
7a702d8
[SPARK-21616][SPARKR][DOCS] update R migration guide and vignettes
felixcheung Jan 1, 2018
c284c4e
[MINOR] Fix a bunch of typos
srowen Dec 31, 2017
1c9f95c
[SPARK-22530][PYTHON][SQL] Adding Arrow support for ArrayType
BryanCutler Jan 1, 2018
e734a4b
[SPARK-21893][SPARK-22142][TESTS][FOLLOWUP] Enables PySpark tests for…
HyukjinKwon Jan 1, 2018
e0c090f
[SPARK-22932][SQL] Refactor AnalysisContext
gatorsmile Jan 2, 2018
a6fc300
[SPARK-22897][CORE] Expose stageAttemptId in TaskContext
advancedxy Jan 2, 2018
247a089
[SPARK-22938] Assert that SQLConf.get is accessed only on the driver.
juliuszsompolski Jan 3, 2018
1a87a16
[SPARK-22934][SQL] Make optional clauses order insensitive for CREATE…
gatorsmile Jan 3, 2018
a66fe36
[SPARK-20236][SQL] dynamic partition overwrite
cloud-fan Jan 3, 2018
9a2b65a
[SPARK-22896] Improvement in String interpolation
chetkhatri Jan 3, 2018
b297029
[SPARK-20960][SQL] make ColumnVector public
cloud-fan Jan 3, 2018
7d045c5
[SPARK-22944][SQL] improve FoldablePropagation
cloud-fan Jan 4, 2018
df95a90
[SPARK-22933][SPARKR] R Structured Streaming API for withWatermark, t…
felixcheung Jan 4, 2018
9fa703e
[SPARK-22950][SQL] Handle ChildFirstURLClassLoader's parent
yaooqinn Jan 4, 2018
d5861ab
[SPARK-22945][SQL] add java UDF APIs in the functions object
cloud-fan Jan 4, 2018
5aadbc9
[SPARK-22939][PYSPARK] Support Spark UDF in registerFunction
gatorsmile Jan 4, 2018
6f68316
[SPARK-22771][SQL] Add a missing return statement in Concat.checkInpu…
maropu Jan 4, 2018
93f92c0
[SPARK-21475][CORE][2ND ATTEMPT] Change to use NIO's Files API for ex…
jerryshao Jan 4, 2018
d2cddc8
[SPARK-22850][CORE] Ensure queued events are delivered to all event q…
Jan 4, 2018
95f9659
[SPARK-22948][K8S] Move SparkPodInitContainer to correct package.
Jan 4, 2018
e288fc8
[SPARK-22953][K8S] Avoids adding duplicated secret volumes when init-…
liyinan926 Jan 4, 2018
0428368
[SPARK-22960][K8S] Make build-push-docker-images.sh more dev-friendly.
Jan 5, 2018
df7fc3e
[SPARK-22957] ApproxQuantile breaks if the number of rows exceeds MaxInt
juliuszsompolski Jan 5, 2018
52fc5c1
[SPARK-22825][SQL] Fix incorrect results of Casting Array to String
maropu Jan 5, 2018
cf0aa65
[SPARK-22949][ML] Apply CrossValidator approach to Driver/Distributed…
MrBago Jan 5, 2018
6cff7d1
[SPARK-22757][K8S] Enable spark.jars and spark.files in KUBERNETES mode
liyinan926 Jan 5, 2018
51c33bd
[SPARK-22961][REGRESSION] Constant columns should generate QueryPlanC…
adrian-ionescu Jan 5, 2018
c0b7424
[SPARK-22940][SQL] HiveExternalCatalogVersionsSuite should succeed on…
bersprockets Jan 5, 2018
930b90a
[SPARK-13030][ML] Follow-up cleanups for OneHotEncoderEstimator
jkbradley Jan 5, 2018
ea95683
[SPARK-22914][DEPLOY] Register history.ui.port
gerashegalov Jan 6, 2018
e8af7e8
[SPARK-22937][SQL] SQL elt output binary for binary inputs
maropu Jan 6, 2018
bf65cd3
[SPARK-22960][K8S] Revert use of ARG base_image in images
liyinan926 Jan 6, 2018
f2dd8b9
[SPARK-22930][PYTHON][SQL] Improve the description of Vectorized UDFs…
icexelloss Jan 6, 2018
be9a804
[SPARK-22793][SQL] Memory leak in Spark Thrift Server
Jan 6, 2018
7b78041
[SPARK-21786][SQL] When acquiring 'compressionCodecClassName' in 'Par…
fjh100456 Jan 6, 2018
993f215
[SPARK-22901][PYTHON][FOLLOWUP] Adds the doc for asNondeterministic f…
HyukjinKwon Jan 6, 2018
9a7048b
[HOTFIX] Fix style checking failure
gatorsmile Jan 6, 2018
18e9414
[SPARK-22973][SQL] Fix incorrect results of Casting Map to String
maropu Jan 7, 2018
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-22397][ML] add multiple columns support to QuantileDiscretizer
## What changes were proposed in this pull request?

add multi columns support to  QuantileDiscretizer.
When calculating the splits, we can either merge together all the  probabilities into one array by calculating approxQuantiles on multiple columns at once, or compute approxQuantiles separately  for each column. After doing the performance comparision, we found it’s better to calculating approxQuantiles on multiple columns at once.

Here is how we measuring the performance time:
```
    var duration = 0.0
    for (i<- 0 until 10) {
      val start = System.nanoTime()
      discretizer.fit(df)
      val end = System.nanoTime()
      duration += (end - start) / 1e9
    }
    println(duration/10)
```
Here is the performance test result:

|numCols |NumRows  | compute each approxQuantiles separately|compute multiple columns approxQuantiles at one time|
|--------|----------|--------------------------------|-------------------------------------------|
|10         |60             |0.3623195839                            |0.1626658607                                                |
|10         |6000        |0.7537239841                             |0.3869370046                                               |
|22         |6000        |1.6497598557                             |0.4767903059                                               |
|50         |6000        |3.2268305752                            |0.7217818396                                                |

## How was this patch tested?

add UT in QuantileDiscretizerSuite to test multi columns supports

Author: Huaxin Gao <[email protected]>

Closes apache#19715 from huaxingao/spark_22397.
  • Loading branch information
huaxingao authored and Nick Pentreath committed Dec 31, 2017
commit 3d8837e59aadd726805371041567ceff375194c0
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.ml._
import org.apache.spark.ml.attribute.NominalAttribute
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasOutputCol}
import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasInputCols, HasOutputCol, HasOutputCols}
import org.apache.spark.ml.util._
import org.apache.spark.sql.Dataset
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -50,10 +50,28 @@ private[feature] trait QuantileDiscretizerBase extends Params
/** @group getParam */
def getNumBuckets: Int = getOrDefault(numBuckets)

/**
* Array of number of buckets (quantiles, or categories) into which data points are grouped.
* Each value must be greater than or equal to 2
*
* See also [[handleInvalid]], which can optionally create an additional bucket for NaN values.
*
* @group param
*/
val numBucketsArray = new IntArrayParam(this, "numBucketsArray", "Array of number of buckets " +
"(quantiles, or categories) into which data points are grouped. This is for multiple " +
"columns input. If transforming multiple columns and numBucketsArray is not set, but " +
"numBuckets is set, then numBuckets will be applied across all columns.",
(arrayOfNumBuckets: Array[Int]) => arrayOfNumBuckets.forall(ParamValidators.gtEq(2)))

/** @group getParam */
def getNumBucketsArray: Array[Int] = $(numBucketsArray)

/**
* Relative error (see documentation for
* `org.apache.spark.sql.DataFrameStatFunctions.approxQuantile` for description)
* Must be in the range [0, 1].
* Note that in multiple columns case, relative error is applied to all columns.
* default: 0.001
* @group param
*/
Expand All @@ -68,7 +86,9 @@ private[feature] trait QuantileDiscretizerBase extends Params
/**
* Param for how to handle invalid entries. Options are 'skip' (filter out rows with
* invalid values), 'error' (throw an error), or 'keep' (keep invalid values in a special
* additional bucket).
* additional bucket). Note that in the multiple columns case, the invalid handling is applied
* to all columns. That said for 'error' it will throw an error if any invalids are found in
* any column, for 'skip' it will skip rows with any invalids in any columns, etc.
* Default: "error"
* @group param
*/
Expand All @@ -86,6 +106,11 @@ private[feature] trait QuantileDiscretizerBase extends Params
* categorical features. The number of bins can be set using the `numBuckets` parameter. It is
* possible that the number of buckets used will be smaller than this value, for example, if there
* are too few distinct values of the input to create enough distinct quantiles.
* Since 2.3.0, `QuantileDiscretizer` can map multiple columns at once by setting the `inputCols`
* parameter. If both of the `inputCol` and `inputCols` parameters are set, an Exception will be
* thrown. To specify the number of buckets for each column, the `numBucketsArray` parameter can
* be set, or if the number of buckets should be the same across columns, `numBuckets` can be
* set as a convenience.
*
* NaN handling:
* null and NaN values will be ignored from the column during `QuantileDiscretizer` fitting. This
Expand All @@ -104,7 +129,8 @@ private[feature] trait QuantileDiscretizerBase extends Params
*/
@Since("1.6.0")
final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val uid: String)
extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable {
extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable
with HasInputCols with HasOutputCols {

@Since("1.6.0")
def this() = this(Identifiable.randomUID("quantileDiscretizer"))
Expand All @@ -129,34 +155,96 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui
@Since("2.1.0")
def setHandleInvalid(value: String): this.type = set(handleInvalid, value)

/** @group setParam */
@Since("2.3.0")
def setNumBucketsArray(value: Array[Int]): this.type = set(numBucketsArray, value)

/** @group setParam */
@Since("2.3.0")
def setInputCols(value: Array[String]): this.type = set(inputCols, value)

/** @group setParam */
@Since("2.3.0")
def setOutputCols(value: Array[String]): this.type = set(outputCols, value)

private[feature] def getInOutCols: (Array[String], Array[String]) = {
require((isSet(inputCol) && isSet(outputCol) && !isSet(inputCols) && !isSet(outputCols)) ||
(!isSet(inputCol) && !isSet(outputCol) && isSet(inputCols) && isSet(outputCols)),
"QuantileDiscretizer only supports setting either inputCol/outputCol or" +
"inputCols/outputCols."
)

if (isSet(inputCol)) {
(Array($(inputCol)), Array($(outputCol)))
} else {
require($(inputCols).length == $(outputCols).length,
"inputCols number do not match outputCols")
($(inputCols), $(outputCols))
}
}

@Since("1.6.0")
override def transformSchema(schema: StructType): StructType = {
SchemaUtils.checkNumericType(schema, $(inputCol))
val inputFields = schema.fields
require(inputFields.forall(_.name != $(outputCol)),
s"Output column ${$(outputCol)} already exists.")
val attr = NominalAttribute.defaultAttr.withName($(outputCol))
val outputFields = inputFields :+ attr.toStructField()
val (inputColNames, outputColNames) = getInOutCols
val existingFields = schema.fields
var outputFields = existingFields
inputColNames.zip(outputColNames).foreach { case (inputColName, outputColName) =>
SchemaUtils.checkNumericType(schema, inputColName)
require(existingFields.forall(_.name != outputColName),
s"Output column ${outputColName} already exists.")
val attr = NominalAttribute.defaultAttr.withName(outputColName)
outputFields :+= attr.toStructField()
}
StructType(outputFields)
}

@Since("2.0.0")
override def fit(dataset: Dataset[_]): Bucketizer = {
transformSchema(dataset.schema, logging = true)
val splits = dataset.stat.approxQuantile($(inputCol),
(0.0 to 1.0 by 1.0/$(numBuckets)).toArray, $(relativeError))
val bucketizer = new Bucketizer(uid).setHandleInvalid($(handleInvalid))
if (isSet(inputCols)) {
val splitsArray = if (isSet(numBucketsArray)) {
val probArrayPerCol = $(numBucketsArray).map { numOfBuckets =>
(0.0 to 1.0 by 1.0 / numOfBuckets).toArray
}

val probabilityArray = probArrayPerCol.flatten.sorted.distinct
val splitsArrayRaw = dataset.stat.approxQuantile($(inputCols),
probabilityArray, $(relativeError))

splitsArrayRaw.zip(probArrayPerCol).map { case (splits, probs) =>
val probSet = probs.toSet
val idxSet = probabilityArray.zipWithIndex.collect {
case (p, idx) if probSet(p) =>
idx
}.toSet
splits.zipWithIndex.collect {
case (s, idx) if idxSet(idx) =>
s
}
}
} else {
dataset.stat.approxQuantile($(inputCols),
(0.0 to 1.0 by 1.0 / $(numBuckets)).toArray, $(relativeError))
}
bucketizer.setSplitsArray(splitsArray.map(getDistinctSplits))
} else {
val splits = dataset.stat.approxQuantile($(inputCol),
(0.0 to 1.0 by 1.0 / $(numBuckets)).toArray, $(relativeError))
bucketizer.setSplits(getDistinctSplits(splits))
}
copyValues(bucketizer.setParent(this))
}

private def getDistinctSplits(splits: Array[Double]): Array[Double] = {
splits(0) = Double.NegativeInfinity
splits(splits.length - 1) = Double.PositiveInfinity

val distinctSplits = splits.distinct
if (splits.length != distinctSplits.length) {
log.warn(s"Some quantiles were identical. Bucketing to ${distinctSplits.length - 1}" +
s" buckets as a result.")
}
val bucketizer = new Bucketizer(uid)
.setSplits(distinctSplits.sorted)
.setHandleInvalid($(handleInvalid))
copyValues(bucketizer.setParent(this))
distinctSplits.sorted
}

@Since("1.6.0")
Expand Down
Loading