Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-37178][ML] Add Target Encoding to ml.feature #48347

Closed
wants to merge 18 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
18 commits
Select commit Hold shift + click to select a range
5715092
[SPARK-37178][ML] Add Target Encoding to ml.feature
Oct 4, 2024
5f1902e
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 4, 2024
0264264
[SPARK-37178][ML] handle null category, support all numeric types, im…
Oct 8, 2024
12cba08
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 8, 2024
0221933
[SPARK-37178][ML] ignore null label observations
Oct 9, 2024
a329d16
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 9, 2024
3f1f86d
[SPARK-37178][ML] improve doc & comments
Oct 16, 2024
cc3b78c
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 16, 2024
229e5ed
[SPARK-37178][ML] allow different feature names in model
Oct 20, 2024
5a67c50
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 20, 2024
bb95c8d
[SPARK-37178][ML] passing raw stats to model, building encodings in t…
Oct 23, 2024
8bd8527
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 23, 2024
32adb85
[SPARK-37178][ML] disregard NaN-labeled observations
Oct 28, 2024
1796454
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Oct 28, 2024
fea4b6c
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Nov 5, 2024
7ca04f3
nits
HyukjinKwon Nov 5, 2024
6236bd0
[SPARK-37178][ML] changed category datatype to Double
Nov 6, 2024
26410a1
Merge branch 'master' of https://github.com/rebo16v/spark-target-enco…
Nov 6, 2024
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
110 changes: 110 additions & 0 deletions docs/ml-features.md
Original file line number Diff line number Diff line change
Expand Up @@ -855,6 +855,116 @@ for more details on the API.

</div>

## TargetEncoder

[Target Encoding](https://www.researchgate.net/publication/220520258_A_Preprocessing_Scheme_for_High-Cardinality_Categorical_Attributes_in_Classification_and_Prediction_Problems) is a data-preprocessing technique that transforms high-cardinality categorical features into quasi-continuous scalar attributes suited for use in regression-type models. This paradigm maps individual values of an independent feature to a scalar, representing some estimate of the dependent attribute (meaning categorical values that exhibit similar statistics with respect to the target will have a similar representation).

By leveraging the relationship between categorical features and the target variable, Target Encoding usually performs better than One-Hot and does not require a final binary vector encoding, decreasing the overall dimensionality of the dataset.

User can specify input and output column names by setting `inputCol` and `outputCol` for single-column use cases, or `inputCols` and `outputCols` for multi-column use cases (both arrays required to have the same size). These columns are expected to contain categorical indices (positive integers), being missing values (null) treated as a separate category. Data type must be any subclass of 'NumericType'. For string type input data, it is common to encode categorical features using [StringIndexer](ml-features.html#stringindexer) first.

User can specify the target column name by setting `label`. This column is expected to contain the ground-truth labels from which encodings will be derived. Observations with missing label (null) are not considered when calculating estimates. Data type must be any subclass of 'NumericType'.

`TargetEncoder` supports the `handleInvalid` parameter to choose how to handle invalid input, meaning categories not seen at training, when encoding new data. Available options include 'keep' (any invalid inputs are assigned to an extra categorical index) and 'error' (throw an exception).

`TargetEncoder` supports the `targetType` parameter to choose the label type when fitting data, affecting how estimates are calculated. Available options include 'binary' and 'continuous'.

When set to 'binary', the target attribute $Y$ is expected to be binary, $Y\in\{ 0,1 \}$. The transformation maps individual values $X_{i}$ to the conditional probability of $Y$ given that $X=X_{i}\;$: $\;\; S_{i}=P(Y\mid X=X_{i})$. This approach is also known as bin-counting.

When set to 'continuous', the target attribute $Y$ is expected to be continuous, $Y\in\mathbb{Q}$. The transformation maps individual values $X_{i}$ to the average of $Y$ given that $X=X_{i}\;$: $\;\; S_{i}=E[Y\mid X=X_{i}]$. This approach is also known as mean-encoding.

`TargetEncoder` supports the `smoothing` parameter to tune how in-category stats and overall stats are blended. High-cardinality categorical features are usually unevenly distributed across all possible values of $X$.
Therefore, calculating encodings $S_{i}$ according only to in-class statistics makes this estimates very unreliable, and rarely seen categories will very likely cause overfitting in learning.

Smoothing prevents this behaviour by weighting in-class estimates with overall estimates according to the relative size of the particular class on the whole dataset.

$\;\;\; S_{i}=\lambda(n_{i})\, P(Y\mid X=X_{i})+(1-\lambda(n_{i}))\, P(Y)$ for the binary case

$\;\;\; S_{i}=\lambda(n_{i})\, E[Y\mid X=X_{i}]+(1-\lambda(n_{i}))\, E[Y]$ for the continuous case

being $\lambda(n_{i})$ a monotonically increasing function on $n_{i}$, bounded between 0 and 1.

Usually $\lambda(n_{i})$ is implemented as the parametric function $\lambda(n_{i})=\frac{n_{i}}{n_{i}+m}$, where $m$ is the smoothing factor, represented by `smoothing` parameter in `TargetEncoder`.

**Examples**

Building on the `TargetEncoder` example, let's assume we have the following
DataFrame with columns `feature` and `target` (binary & continuous):

~~~~
feature | target | target
| (bin) | (cont)
--------|--------|--------
1 | 0 | 1.3
1 | 1 | 2.5
1 | 0 | 1.6
2 | 1 | 1.8
2 | 0 | 2.4
3 | 1 | 3.2
~~~~

Applying `TargetEncoder` with 'binary' target type,
`feature` as the input column,`target (bin)` as the label column
and `encoded` as the output column, we are able to fit a model
on the data to learn encodings and transform the data according
to these mappings:

~~~~
feature | target | encoded
| (bin) |
--------|--------|--------
1 | 0 | 0.333
1 | 1 | 0.333
1 | 0 | 0.333
2 | 1 | 0.5
2 | 0 | 0.5
3 | 1 | 1.0
~~~~

Applying `TargetEncoder` with 'continuous' target type,
`feature` as the input column,`target (cont)` as the label column
and `encoded` as the output column, we are able to fit a model
on the data to learn encodings and transform the data according
to these mappings:

~~~~
feature | target | encoded
| (cont) |
--------|--------|--------
1 | 1.3 | 1.8
1 | 2.5 | 1.8
1 | 1.6 | 1.8
2 | 1.8 | 2.1
2 | 2.4 | 2.1
3 | 3.2 | 3.2
~~~~

<div class="codetabs">

<div data-lang="python" markdown="1">

Refer to the [TargetEncoder Python docs](api/python/reference/api/pyspark.ml.feature.TargetEncoder.html) for more details on the API.

{% include_example python/ml/target_encoder_example.py %}
</div>

<div data-lang="scala" markdown="1">

Refer to the [TargetEncoder Scala docs](api/scala/org/apache/spark/ml/feature/TargetEncoder.html) for more details on the API.

{% include_example scala/org/apache/spark/examples/ml/TargetEncoderExample.scala %}
</div>

<div data-lang="java" markdown="1">

Refer to the [TargetEncoder Java docs](api/java/org/apache/spark/ml/feature/TargetEncoder.html)
for more details on the API.

{% include_example java/org/apache/spark/examples/ml/JavaTargetEncoderExample.java %}
</div>

</div>

## VectorIndexer

`VectorIndexer` helps index categorical features in datasets of `Vector`s.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
/*
* 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.examples.ml;

import org.apache.spark.sql.SparkSession;

// $example on$
import org.apache.spark.ml.feature.TargetEncoder;
import org.apache.spark.ml.feature.TargetEncoderModel;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.Metadata;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;

import java.util.Arrays;
import java.util.List;
// $example off$

public class JavaTargetEncoderExample {
public static void main(String[] args) {
SparkSession spark = SparkSession
.builder()
.appName("JavaTargetEncoderExample")
.getOrCreate();

// Note: categorical features are usually first encoded with StringIndexer
// $example on$
List<Row> data = Arrays.asList(
RowFactory.create(0.0, 1.0, 0, 10.0),
RowFactory.create(1.0, 0.0, 1, 20.0),
RowFactory.create(2.0, 1.0, 0, 30.0),
RowFactory.create(0.0, 2.0, 1, 40.0),
RowFactory.create(0.0, 1.0, 0, 50.0),
RowFactory.create(2.0, 0.0, 1, 60.0)
);

StructType schema = new StructType(new StructField[]{
new StructField("categoryIndex1", DataTypes.DoubleType, false, Metadata.empty()),
new StructField("categoryIndex2", DataTypes.DoubleType, false, Metadata.empty()),
new StructField("binaryLabel", DataTypes.DoubleType, false, Metadata.empty()),
new StructField("continuousLabel", DataTypes.DoubleType, false, Metadata.empty())
});

Dataset<Row> df = spark.createDataFrame(data, schema);

// binary target
TargetEncoder bin_encoder = new TargetEncoder()
.setInputCols(new String[] {"categoryIndex1", "categoryIndex2"})
.setOutputCols(new String[] {"categoryIndex1Target", "categoryIndex2Target"})
.setLabelCol("binaryLabel")
.setTargetType("binary");

TargetEncoderModel bin_model = bin_encoder.fit(df);
Dataset<Row> bin_encoded = bin_model.transform(df);
bin_encoded.show();

// continuous target
TargetEncoder cont_encoder = new TargetEncoder()
.setInputCols(new String[] {"categoryIndex1", "categoryIndex2"})
.setOutputCols(new String[] {"categoryIndex1Target", "categoryIndex2Target"})
.setLabelCol("continuousLabel")
.setTargetType("continuous");

TargetEncoderModel cont_model = cont_encoder.fit(df);
Dataset<Row> cont_encoded = cont_model.transform(df);
cont_encoded.show();
// $example off$

spark.stop();
}
}

65 changes: 65 additions & 0 deletions examples/src/main/python/ml/target_encoder_example.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
#
# 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.
#

# $example on$
from pyspark.ml.feature import TargetEncoder

# $example off$
from pyspark.sql import SparkSession

if __name__ == "__main__":
spark = SparkSession.builder.appName("TargetEncoderExample").getOrCreate()

# Note: categorical features are usually first encoded with StringIndexer
# $example on$
df = spark.createDataFrame(
[
(0.0, 1.0, 0, 10.0),
(1.0, 0.0, 1, 20.0),
(2.0, 1.0, 0, 30.0),
(0.0, 2.0, 1, 40.0),
(0.0, 1.0, 0, 50.0),
(2.0, 0.0, 1, 60.0),
],
["categoryIndex1", "categoryIndex2", "binaryLabel", "continuousLabel"],
)

# binary target
encoder = TargetEncoder(
inputCols=["categoryIndex1", "categoryIndex2"],
outputCols=["categoryIndex1Target", "categoryIndex2Target"],
labelCol="binaryLabel",
targetType="binary"
)
model = encoder.fit(df)
encoded = model.transform(df)
encoded.show()

# continuous target
encoder = TargetEncoder(
inputCols=["categoryIndex1", "categoryIndex2"],
outputCols=["categoryIndex1Target", "categoryIndex2Target"],
labelCol="continuousLabel",
targetType="continuous"
)

model = encoder.fit(df)
encoded = model.transform(df)
encoded.show()
# $example off$

spark.stop()
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.
*/

// scalastyle:off println
package org.apache.spark.examples.ml

// $example on$
import org.apache.spark.ml.feature.TargetEncoder
// $example off$
import org.apache.spark.sql.SparkSession

object TargetEncoderExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
.builder()
.appName("TargetEncoderExample")
.getOrCreate()

// Note: categorical features are usually first encoded with StringIndexer
// $example on$
val df = spark.createDataFrame(Seq(
(0.0, 1.0, 0, 10.0),
(1.0, 0.0, 1, 20.0),
(2.0, 1.0, 0, 30.0),
(0.0, 2.0, 1, 40.0),
(0.0, 1.0, 0, 50.0),
(2.0, 0.0, 1, 60.0)
)).toDF("categoryIndex1", "categoryIndex2",
"binaryLabel", "continuousLabel")

// binary target
val bin_encoder = new TargetEncoder()
.setInputCols(Array("categoryIndex1", "categoryIndex2"))
.setOutputCols(Array("categoryIndex1Target", "categoryIndex2Target"))
.setLabelCol("binaryLabel")
.setTargetType("binary");

val bin_model = bin_encoder.fit(df)
val bin_encoded = bin_model.transform(df)
bin_encoded.show()

// continuous target
val cont_encoder = new TargetEncoder()
.setInputCols(Array("categoryIndex1", "categoryIndex2"))
.setOutputCols(Array("categoryIndex1Target", "categoryIndex2Target"))
.setLabelCol("continuousLabel")
.setTargetType("continuous");

val cont_model = cont_encoder.fit(df)
val cont_encoded = cont_model.transform(df)
cont_encoded.show()
// $example off$

spark.stop()
}
}
// scalastyle:on println
Loading