-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23975][ML]Allow Clustering to take Arrays of Double as input features #21081
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
Closed
Closed
Changes from 1 commit
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
ed890d3
add Array input support for KMeans
lu-wang-dl badb0cc
remove redundent code
lu-wang-dl 6d222a3
make sure the code works for Float type and add the unit test
lu-wang-dl 009b918
consolidating featuretovector
lu-wang-dl cd988c7
change featureToVector to KMeanParams and add the scala docs
lu-wang-dl 3ffb322
change featureToVector and validateSchema to private functions
lu-wang-dl fee36ad
Merge branch 'master' of https://github.com/apache/spark into SPARK-2…
lu-wang-dl 3e012fb
move featureToVector to util, so that other methods could use it to a…
lu-wang-dl c4e1a51
fix the comments and correct code style
lu-wang-dl File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,6 @@ | |
|
|
||
| package org.apache.spark.ml.util | ||
|
|
||
| import org.apache.spark.annotation.Since | ||
| import org.apache.spark.ml.linalg.{Vectors, VectorUDT} | ||
| import org.apache.spark.sql.{Column, Dataset} | ||
| import org.apache.spark.sql.functions.{col, udf} | ||
|
|
@@ -27,28 +26,38 @@ import org.apache.spark.sql.types.{ArrayType, DoubleType, FloatType} | |
| private[spark] object DatasetUtils { | ||
|
|
||
| /** | ||
| * preprocessing the input feature column to Vector | ||
| * @param dataset DataFrame with columns for features | ||
| * @param colName column name for features | ||
| * @return Vector feature column | ||
| * Cast a column in a Dataset to Vector type. | ||
| * | ||
| * The supported data types of the input column are | ||
| * - Vector | ||
| * - float/double type Array. | ||
| * | ||
| * Note: The returned column does not have Metadata. | ||
| * | ||
| * @param dataset input DataFrame | ||
| * @param colName column name. | ||
| * @return Vector column | ||
| */ | ||
| @Since("2.4.0") | ||
| def columnToVector(dataset: Dataset[_], colName: String): Column = { | ||
| val featuresDataType = dataset.schema(colName).dataType | ||
| featuresDataType match { | ||
| val columnDataType = dataset.schema(colName).dataType | ||
| columnDataType match { | ||
| case _: VectorUDT => col(colName) | ||
| case fdt: ArrayType => | ||
| val transferUDF = fdt.elementType match { | ||
| case _: FloatType => udf(f = (vector: Seq[Float]) => { | ||
| val featureArray = Array.fill[Double](vector.size)(0.0) | ||
| vector.indices.foreach(idx => featureArray(idx) = vector(idx).toDouble) | ||
| Vectors.dense(featureArray) | ||
| val inputArray = Array.fill[Double](vector.size)(0.0) | ||
| vector.indices.foreach(idx => inputArray(idx) = vector(idx).toDouble) | ||
| Vectors.dense(inputArray) | ||
| }) | ||
| case _: DoubleType => udf((vector: Seq[Double]) => { | ||
| Vectors.dense(vector.toArray) | ||
| }) | ||
| case other => | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks! I forgot about this since this was generalized. |
||
| throw new IllegalArgumentException(s"Array[$other] column cannot be cast to Vector") | ||
| } | ||
| transferUDF(col(colName)) | ||
| case other => | ||
| throw new IllegalArgumentException(s"$other column cannot be cast to Vector") | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
scala style: always put newline between methods
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ping: There needs to be a newline between the "}" of the previous method and the "/**" Scaladoc of the next method. Please start checking for this.