-
Notifications
You must be signed in to change notification settings - Fork 29k
[Spark-18080][ML][PYTHON] Python API & Examples for Locality Sensitive Hashing #16715
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
Changes from 24 commits
85d22c3
cdeca1c
66d308b
d62a2d0
dafc4d1
ac1f4f7
3a21f26
65dab3e
3d3bcf0
69dccde
e7542d0
5cfc9c5
ccabbf4
2508a2f
2dd6aad
8e5468f
6e85e1a
4bc670c
b45ec0a
1b70b91
b1da01e
8f1d708
49edc93
c64d50b
5d55752
d849c3a
36fd9bc
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
|
|
||
| package org.apache.spark.examples.ml; | ||
|
|
||
| import org.apache.spark.ml.linalg.Vector; | ||
|
||
| import org.apache.spark.sql.SparkSession; | ||
|
|
||
| // $example on$ | ||
|
|
@@ -34,8 +35,15 @@ | |
| import org.apache.spark.sql.types.Metadata; | ||
| import org.apache.spark.sql.types.StructField; | ||
| import org.apache.spark.sql.types.StructType; | ||
|
|
||
| import static org.apache.spark.sql.functions.col; | ||
| // $example off$ | ||
|
|
||
| /** | ||
| * An example demonstrating MinHashLSH. | ||
| * Run with: | ||
| * bin/run-example org.apache.spark.examples.ml.JavaMinHashLSHExample | ||
| */ | ||
| public class JavaMinHashLSHExample { | ||
| public static void main(String[] args) { | ||
| SparkSession spark = SparkSession | ||
|
|
@@ -44,25 +52,58 @@ public static void main(String[] args) { | |
| .getOrCreate(); | ||
|
|
||
| // $example on$ | ||
| List<Row> data = Arrays.asList( | ||
| List<Row> dataA = Arrays.asList( | ||
| RowFactory.create(0, Vectors.sparse(6, new int[]{0, 1, 2}, new double[]{1.0, 1.0, 1.0})), | ||
| RowFactory.create(1, Vectors.sparse(6, new int[]{2, 3, 4}, new double[]{1.0, 1.0, 1.0})), | ||
| RowFactory.create(2, Vectors.sparse(6, new int[]{0, 2, 4}, new double[]{1.0, 1.0, 1.0})) | ||
| ); | ||
|
|
||
| List<Row> dataB = Arrays.asList( | ||
| RowFactory.create(0, Vectors.sparse(6, new int[]{1, 3, 5}, new double[]{1.0, 1.0, 1.0})), | ||
| RowFactory.create(1, Vectors.sparse(6, new int[]{2, 3, 5}, new double[]{1.0, 1.0, 1.0})), | ||
| RowFactory.create(2, Vectors.sparse(6, new int[]{1, 2, 4}, new double[]{1.0, 1.0, 1.0})) | ||
| ); | ||
|
|
||
| StructType schema = new StructType(new StructField[]{ | ||
| new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), | ||
| new StructField("keys", new VectorUDT(), false, Metadata.empty()) | ||
| new StructField("features", new VectorUDT(), false, Metadata.empty()) | ||
| }); | ||
| Dataset<Row> dataFrame = spark.createDataFrame(data, schema); | ||
| Dataset<Row> dfA = spark.createDataFrame(dataA, schema); | ||
| Dataset<Row> dfB = spark.createDataFrame(dataB, schema); | ||
|
|
||
| int[] indices = {1, 3}; | ||
| double[] values = {1.0, 1.0}; | ||
| Vector key = Vectors.sparse(6, indices, values); | ||
|
|
||
| MinHashLSH mh = new MinHashLSH() | ||
| .setNumHashTables(1) | ||
| .setInputCol("keys") | ||
| .setOutputCol("values"); | ||
| .setNumHashTables(5) | ||
| .setInputCol("features") | ||
| .setOutputCol("hashes"); | ||
|
|
||
| MinHashLSHModel model = mh.fit(dfA); | ||
|
|
||
| // Feature Transformation | ||
| System.out.println("The hashed dataset where hashed values are stored in the column 'hashes':"); | ||
| model.transform(dfA).show(); | ||
|
|
||
| // Compute the locality sensitive hashes for the input rows, then perform approximate | ||
| // similarity join. | ||
| // We could avoid computing hashes by passing in the already-transformed dataset, e.g. | ||
| // `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` | ||
| System.out.println("Approximately joining dfA and dfB on Jaccard distance smaller than 0.6:"); | ||
| model.approxSimilarityJoin(dfA, dfB, 0.6, "JaccardDistance") | ||
| .select(col("datasetA.id").alias("idA"), | ||
| col("datasetB.id").alias("idB"), | ||
| col("JaccardDistance")).show(); | ||
|
|
||
| MinHashLSHModel model = mh.fit(dataFrame); | ||
| model.transform(dataFrame).show(); | ||
| // Compute the locality sensitive hashes for the input rows, then perform approximate nearest | ||
| // neighbor search. | ||
| // We could avoid computing hashes by passing in the already-transformed dataset, e.g. | ||
| // `model.approxNearestNeighbors(transformedA, key, 2)` | ||
| // It may return less than 2 rows when not enough approximate near-neighbor candidates are | ||
| // found. | ||
| System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); | ||
| model.approxNearestNeighbors(dfA, key, 2).show(); | ||
| // $example off$ | ||
|
|
||
| spark.stop(); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,81 @@ | ||
| # | ||
| # 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. | ||
| # | ||
|
|
||
|
|
||
| from __future__ import print_function | ||
|
|
||
| # $example on$ | ||
| from pyspark.ml.feature import BucketedRandomProjectionLSH | ||
| from pyspark.ml.linalg import Vectors | ||
| from pyspark.sql.functions import col | ||
| # $example off$ | ||
| from pyspark.sql import SparkSession | ||
|
|
||
| """ | ||
| An example demonstrating BucketedRandomProjectionLSH. | ||
| Run with: | ||
| bin/spark-submit examples/src/main/python/ml/bucketed_random_projection_lsh_example.py | ||
|
Contributor
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. Can we add the appropriate note for this to the Scala and Java examples as well?
Contributor
Author
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. Added in 4 places. |
||
| """ | ||
|
|
||
| if __name__ == "__main__": | ||
| spark = SparkSession \ | ||
| .builder \ | ||
| .appName("BucketedRandomProjectionLSHExample") \ | ||
| .getOrCreate() | ||
|
|
||
| # $example on$ | ||
| dataA = [(0, Vectors.dense([1.0, 1.0]),), | ||
| (1, Vectors.dense([1.0, -1.0]),), | ||
| (2, Vectors.dense([-1.0, -1.0]),), | ||
| (3, Vectors.dense([-1.0, 1.0]),)] | ||
| dfA = spark.createDataFrame(dataA, ["id", "features"]) | ||
|
|
||
| dataB = [(4, Vectors.dense([1.0, 0.0]),), | ||
| (5, Vectors.dense([-1.0, 0.0]),), | ||
| (6, Vectors.dense([0.0, 1.0]),), | ||
| (7, Vectors.dense([0.0, -1.0]),)] | ||
| dfB = spark.createDataFrame(dataB, ["id", "features"]) | ||
|
|
||
| key = Vectors.dense([1.0, 0.0]) | ||
|
|
||
| brp = BucketedRandomProjectionLSH(inputCol="features", outputCol="hashes", bucketLength=2.0, | ||
| numHashTables=3) | ||
| model = brp.fit(dfA) | ||
|
|
||
| # Feature Transformation | ||
| print("The hashed dataset where hashed values are stored in the column 'hashes':") | ||
| model.transform(dfA).show() | ||
|
|
||
| # Compute the locality sensitive hashes for the input rows, then perform approximate | ||
| # similarity join. | ||
| # We could avoid computing hashes by passing in the already-transformed dataset, e.g. | ||
| # `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` | ||
| print("Approximately joining dfA and dfB on Euclidean distance smaller than 1.5:") | ||
| model.approxSimilarityJoin(dfA, dfB, 1.5, distCol="EuclideanDistance")\ | ||
| .select(col("datasetA.id").alias("idA"), | ||
| col("datasetB.id").alias("idB"), | ||
| col("EuclideanDistance")).show() | ||
|
|
||
| # Compute the locality sensitive hashes for the input rows, then perform approximate nearest | ||
| # neighbor search. | ||
| # We could avoid computing hashes by passing in the already-transformed dataset, e.g. | ||
| # `model.approxNearestNeighbors(transformedA, key, 2)` | ||
| print("Approximately searching dfA for 2 nearest neighbors of the key:") | ||
| model.approxNearestNeighbors(dfA, key, 2).show() | ||
| # $example off$ | ||
|
|
||
| spark.stop() | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,81 @@ | ||
| # | ||
| # 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. | ||
| # | ||
|
|
||
|
|
||
| from __future__ import print_function | ||
|
|
||
| # $example on$ | ||
| from pyspark.ml.feature import MinHashLSH | ||
| from pyspark.ml.linalg import Vectors | ||
| from pyspark.sql.functions import col | ||
| # $example off$ | ||
| from pyspark.sql import SparkSession | ||
|
|
||
| """ | ||
| An example demonstrating MinHashLSH. | ||
| Run with: | ||
| bin/spark-submit examples/src/main/python/ml/min_hash_lsh_example.py | ||
| """ | ||
|
|
||
| if __name__ == "__main__": | ||
| spark = SparkSession \ | ||
| .builder \ | ||
| .appName("MinHashLSHExample") \ | ||
| .getOrCreate() | ||
|
|
||
| # $example on$ | ||
| dataA = [(0, Vectors.sparse(6, [0, 1, 2], [1.0, 1.0, 1.0]),), | ||
| (1, Vectors.sparse(6, [2, 3, 4], [1.0, 1.0, 1.0]),), | ||
| (2, Vectors.sparse(6, [0, 2, 4], [1.0, 1.0, 1.0]),)] | ||
| dfA = spark.createDataFrame(dataA, ["id", "features"]) | ||
|
|
||
| dataB = [(3, Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), | ||
| (4, Vectors.sparse(6, [2, 3, 5], [1.0, 1.0, 1.0]),), | ||
| (5, Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] | ||
| dfB = spark.createDataFrame(dataB, ["id", "features"]) | ||
|
|
||
| key = Vectors.sparse(6, [1, 3], [1.0, 1.0]) | ||
|
|
||
| mh = MinHashLSH(inputCol="features", outputCol="hashes", numHashTables=5) | ||
| model = mh.fit(dfA) | ||
|
|
||
| # Feature Transformation | ||
| print("The hashed dataset where hashed values are stored in the column 'hashes':") | ||
| model.transform(dfA).show() | ||
|
|
||
| # Compute the locality sensitive hashes for the input rows, then perform approximate | ||
| # similarity join. | ||
| # We could avoid computing hashes by passing in the already-transformed dataset, e.g. | ||
| # `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` | ||
| print("Approximately joining dfA and dfB on distance smaller than 0.6:") | ||
| model.approxSimilarityJoin(dfA, dfB, 0.6, distCol="JaccardDistance")\ | ||
| .select(col("datasetA.id").alias("idA"), | ||
| col("datasetB.id").alias("idB"), | ||
| col("JaccardDistance")).show() | ||
|
|
||
| # Compute the locality sensitive hashes for the input rows, then perform approximate nearest | ||
| # neighbor search. | ||
| # We could avoid computing hashes by passing in the already-transformed dataset, e.g. | ||
| # `model.approxNearestNeighbors(transformedA, key, 2)` | ||
| # It may return less than 2 rows when not enough approximate near-neighbor candidates are | ||
| # found. | ||
| print("Approximately searching dfA for 2 nearest neighbors of the key:") | ||
| model.approxNearestNeighbors(dfA, key, 2).show() | ||
|
|
||
| # $example off$ | ||
|
|
||
| spark.stop() |
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.
Actually we can simplify it as
bin/run-example ml.JavaBucketedRandomProjectionLSHExample, but it's ok to leave as it is.