From 85d22c37d3fe0b907f2eaf892729d087f9efb76c Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 4 Nov 2016 07:22:23 -0700 Subject: [PATCH 01/20] Locality Sensitive Hashing (LSH) Python API. --- .../org/apache/spark/ml/feature/LSH.scala | 26 +- .../org/apache/spark/ml/feature/MinHash.scala | 4 +- .../spark/ml/feature/RandomProjection.scala | 2 +- python/pyspark/ml/feature.py | 309 ++++++++++++++++++ 4 files changed, 325 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index 333a8c364a884..f132272a9bedd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -39,9 +39,9 @@ private[ml] trait LSHParams extends HasInputCol with HasOutputCol { * higher the dimension is, the lower the false negative rate. * @group param */ - final val outputDim: IntParam = new IntParam(this, "outputDim", "output dimension, where" + - "increasing dimensionality lowers the false negative rate, and decreasing dimensionality" + - " improves the running performance", ParamValidators.gt(0)) + final val outputDim: IntParam = new IntParam(this, "outputDim", "The output dimension, where" + + " increasing dimensionality lowers the false negative rate, and decreasing dimensionality" + + " improves the running performance.", ParamValidators.gt(0)) /** @group getParam */ final def getOutputDim: Int = $(outputDim) @@ -109,11 +109,11 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] * - Single Probing: Fast, return at most k elements (Probing only one buckets) * - Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to the key) * - * @param dataset the dataset to search for nearest neighbors of the key - * @param key Feature vector representing the item to search for - * @param numNearestNeighbors The maximum number of nearest neighbors - * @param singleProbing True for using Single Probing; false for multiple probing - * @param distCol Output column for storing the distance between each result row and the key + * @param dataset The dataset to search for nearest neighbors of the key. + * @param key Feature vector representing the item to search for. + * @param numNearestNeighbors The maximum number of nearest neighbors. + * @param singleProbing True for using Single Probing; false for multiple probing. + * @param distCol Output column for storing the distance between each result row and the key. * @return A dataset containing at most k items closest to the key. A distCol is added to show * the distance between each row and the key. */ @@ -215,12 +215,12 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] * [[outputCol]] exists, it will use the [[outputCol]]. This allows caching of the transformed * data when necessary. * - * @param datasetA One of the datasets to join - * @param datasetB Another dataset to join - * @param threshold The threshold for the distance of row pairs - * @param distCol Output column for storing the distance between each result row and the key + * @param datasetA One of the datasets to join. + * @param datasetB Another dataset to join. + * @param threshold The threshold for the distance of row pairs. + * @param distCol Output column for storing the distance between each result row and the key. * @return A joined dataset containing pairs of rows. The original rows are in columns - * "datasetA" and "datasetB", and a distCol is added to show the distance of each pair + * "datasetA" and "datasetB", and a distCol is added to show the distance of each pair. */ def approxSimilarityJoin( datasetA: Dataset[_], diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHash.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHash.scala index d9d0f32254e24..1cc27ab0b2ec5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHash.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHash.scala @@ -33,8 +33,8 @@ import org.apache.spark.sql.types.StructType * * Model produced by [[MinHash]], where multiple hash functions are stored. Each hash function is * a perfect hash function: - * `h_i(x) = (x * k_i mod prime) mod numEntries` - * where `k_i` is the i-th coefficient, and both `x` and `k_i` are from `Z_prime^*` + * `h_i(x) = (x * k_i \mod prime) \mod numEntries` + * where `k_i` is the i-th coefficient, and both `x` and `k_i` are from `Z_{prime^*}` * * Reference: * [[https://en.wikipedia.org/wiki/Perfect_hash_function Wikipedia on Perfect Hash Function]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RandomProjection.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RandomProjection.scala index 1b524c6710b42..af32bdf077709 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RandomProjection.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RandomProjection.scala @@ -60,7 +60,7 @@ private[ml] trait RandomProjectionParams extends Params { * * Model produced by [[RandomProjection]], where multiple random vectors are stored. The vectors * are normalized to be unit vectors and each vector is used in a hash function: - * `h_i(x) = floor(r_i.dot(x) / bucketLength)` + * `h_i(x) = floor(r_i * x / bucketLength)` * where `r_i` is the i-th random unit vector. The number of buckets will be `(max L2 norm of input * vectors) / bucketLength`. * diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 635cf1304588e..37181302d9a2a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -37,6 +37,7 @@ 'IDF', 'IDFModel', 'IndexToString', 'MaxAbsScaler', 'MaxAbsScalerModel', + 'MinHash', 'MinHashModel', 'MinMaxScaler', 'MinMaxScalerModel', 'NGram', 'Normalizer', @@ -44,6 +45,7 @@ 'PCA', 'PCAModel', 'PolynomialExpansion', 'QuantileDiscretizer', + 'RandomProjection', 'RandomProjectionModel', 'RegexTokenizer', 'RFormula', 'RFormulaModel', 'SQLTransformer', @@ -651,6 +653,86 @@ def idf(self): return self._call_java("idf") +class LSHParams(Params): + """ + Mixin for Locality Sensitive Hashing(LSH) algorithm parameters. + """ + + outputDim = Param(Params._dummy(), "outputDim", "The output dimension, where increasing " + + "dimensionality lowers the false negative rate, and decreasing " + + "dimensionality improves the running performance.", + typeConverter=TypeConverters.toInt) + + def __init__(self): + super(LSHParams, self).__init__() + + @since("2.1.0") + def setOutputDim(self, value): + """ + Sets the value of :py:attr:`outputDim`. + """ + return self._set(outputDim=value) + + @since("2.1.0") + def getOutputDim(self): + """ + Gets the value of outputDim or its default value. + """ + return self.getOrDefault(self.outputDim) + + +class LSHModel(): + """ + Mixin for Locality Sensitive Hashing(LSH) models. + """ + + @since("2.1.0") + def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbing=True, + distCol="distCol"): + """ + Given a large dataset and an item, approximately find at most k items which have the + closest distance to the item. If the :py:attr:`outputCol` is missing, the method will + transform the data; if the :py:attr:`outputCol` exists, it will use that. This allows + caching of the transformed data when necessary. + + This method implements two ways of fetching k nearest neighbors: + + * Single Probing: Fast, return at most k elements (Probing only one buckets) + + * Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to the key) + + :param dataset: The dataset to search for nearest neighbors of the key. + :param key: Feature vector representing the item to search for. + :param numNearestNeighbors: The maximum number of nearest neighbors. + :param singleProbing: True for using single probing (default); false for multiple probing. + :param distCol: Output column for storing the distance between each result row and the key. + Use "distCol" as default value if it's not specified. + :return: A dataset containing at most k items closest to the key. A distCol is added + to show the distance between each row and the key. + """ + return self._call_java("approxNearestNeighbors", dataset, key, numNearestNeighbors, + singleProbing, distCol) + + @since("2.1.0") + def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol"): + """ + Join two dataset to approximately find all pairs of rows whose distance are smaller than + the threshold. If the :py:attr:`outputCol` is missing, the method will transform the data; + if the :py:attr:`outputCol` exists, it will use that. This allows caching of the + transformed data when necessary. + + :param datasetA: One of the datasets to join. + :param datasetB: Another dataset to join. + :param threshold: The threshold for the distance of row pairs. + :param distCol: Output column for storing the distance between each result row and the key. + Use "distCol" as default value if it's not specified. + :return: A joined dataset containing pairs of rows. The original rows are in columns + "datasetA" and "datasetB", and a distCol is added to show the distance of + each pair. + """ + return self._call_java("approxSimilarityJoin", datasetA, datasetB, threshold, distCol) + + @inherit_doc class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ @@ -731,6 +813,119 @@ def maxAbs(self): return self._call_java("maxAbs") +@inherit_doc +class MinHash(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, + JavaMLReadable, JavaMLWritable): + + """ + .. note:: Experimental + + LSH class for Jaccard distance. + The input can be dense or sparse vectors, but it is more efficient if it is sparse. + For example, `Vectors.sparse(10, Array[(2, 1.0), (3, 1.0), (5, 1.0)])` + means there are 10 elements in the space. This set contains elem 2, elem 3 and elem 5. + Also, any input vector must have at least 1 non-zero indices, and all non-zero values are treated + as binary "1" values. + + .. seealso:: `MinHash `_ + + >>> from pyspark.ml.linalg import Vectors + >>> data = [(Vectors.sparse(10, [0, 1], [1.0, 1.0]),), + ... (Vectors.sparse(10, [1, 2], [1.0, 1.0]),), + ... (Vectors.sparse(10, [2, 3], [1.0, 1.0]),), + ... (Vectors.sparse(10, [3, 4], [1.0, 1.0]),), + ... (Vectors.sparse(10, [4, 5], [1.0, 1.0]),)] + >>> df = spark.createDataFrame(data, ["keys"]) + >>> mh = MinHash(inputCol="keys", outputCol="values", seed=12345) + >>> model = mh.fit(df) + >>> model.numEntries + 20 + >>> model.randCoefficients + [776966252] + >>> model.transform(df).head() + Row(keys=SparseVector(10, {0: 1.0, 1: 1.0}), values=DenseVector([4.0])) + >>> data2 = [(Vectors.sparse(10, [5, 6], [1.0, 1.0]),), + ... (Vectors.sparse(10, [6, 7], [1.0, 1.0]),), + ... (Vectors.sparse(10, [7, 8], [1.0, 1.0]),), + ... (Vectors.sparse(10, [8, 9], [1.0, 1.0]),)] + >>> df2 = spark.createDataFrame(data2, ["keys"]) + >>> model.approxNearestNeighbors(df2, Vectors.sparse(10, [5, 8], [1.0, 1.0]), 1).collect() + [Row(keys=SparseVector(10, {5: 1.0, 6: 1.0}), values=DenseVector([6.0]), distCol=0.666...)] + >>> model.approxSimilarityJoin(df, df2, 1.0).select("distCol").head()[0] + 0.666... + >>> mhPath = temp_path + "/mh" + >>> mh.save(mhPath) + >>> mh2 = MinHash.load(mhPath) + >>> mh2.getOutputCol() == mh.getOutputCol() + True + >>> modelPath = temp_path + "/mh-model" + >>> model.save(modelPath) + >>> model2 = MinHashModel.load(modelPath) + >>> model2.numEntries == model.numEntries + True + >>> model2.randCoefficients == model.randCoefficients + True + + .. versionadded:: 2.1.0 + """ + + @keyword_only + def __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1): + """ + __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1) + """ + super(MinHash, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinHash", self.uid) + self._setDefault(outputDim=1) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + @since("2.1.0") + def setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1): + """ + setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1) + Sets params for this MinHash. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def _create_model(self, java_model): + return MinHashModel(java_model) + + +class MinHashModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): + """ + .. note:: Experimental + + Model produced by :py:class:`MinHash`, where multiple hash functions are stored. + Each hash function is a perfect hash function: + :math:`h_i(x) = (x * k_i \mod prime) \mod numEntries` + where :math:`k_i` is the i-th coefficient, and both `x` and :math:`k_i` + are from :math:`Z_{prime^*}` + + .. seealso:: `Perfect Hash Function `_ + + .. versionadded:: 2.1.0 + """ + + @property + @since("2.1.0") + def numEntries(self): + """ + The number of entries of the hash functions. + """ + return self._call_java("numEntries") + + @property + @since("2.1.0") + def randCoefficients(self): + """ + An array of random coefficients, each used by one hash function. + """ + return self._call_java("randCoefficients") + + @inherit_doc class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ @@ -1253,6 +1448,120 @@ def _create_model(self, java_model): outputCol=self.getOutputCol()) +@inherit_doc +class RandomProjection(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, + JavaMLReadable, JavaMLWritable): + """ + .. note:: Experimental + + LSH class for Euclidean distance metrics. + The input is dense or sparse vectors, each of which represents a point in the Euclidean + distance space. The output will be vectors of configurable dimension. Hash value in the + same dimension is calculated by the same hash function. + + .. seealso:: `Stable Distributions `_ + .. seealso:: `Hashing for Similarity Search: A Survey `_ + + >>> from pyspark.ml.linalg import Vectors + >>> data = [(Vectors.dense([-1.0, -1.0 ]),), + ... (Vectors.dense([-1.0, 1.0 ]),), + ... (Vectors.dense([1.0, -1.0 ]),), + ... (Vectors.dense([1.0, 1.0]),)] + >>> df = spark.createDataFrame(data, ["keys"]) + >>> rp = RandomProjection(inputCol="keys", outputCol="values", seed=12345, bucketLength=1.0) + >>> model = rp.fit(df) + >>> model.randUnitVectors + [DenseVector([-0.3041, 0.9527])] + >>> model.transform(df).head() + Row(keys=DenseVector([-1.0, -1.0]), values=DenseVector([-1.0])) + >>> data2 = [(Vectors.dense([2.0, 2.0 ]),), + ... (Vectors.dense([2.0, 3.0 ]),), + ... (Vectors.dense([3.0, 2.0 ]),), + ... (Vectors.dense([3.0, 3.0]),)] + >>> df2 = spark.createDataFrame(data2, ["keys"]) + >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() + [Row(keys=DenseVector([2.0, 2.0]), values=DenseVector([1.0]), distCol=1.0)] + >>> model.approxSimilarityJoin(df, df2, 3.0).select("distCol").head()[0] + 2.236... + >>> rpPath = temp_path + "/rp" + >>> rp.save(rpPath) + >>> rp2 = RandomProjection.load(rpPath) + >>> rp2.getBucketLength() == rp.getBucketLength() + True + >>> modelPath = temp_path + "/rp-model" + >>> model.save(modelPath) + >>> model2 = RandomProjectionModel.load(modelPath) + >>> model2.randUnitVectors == model.randUnitVectors + True + + .. versionadded:: 2.1.0 + """ + + bucketLength = Param(Params._dummy(), "bucketLength", "the length of each hash bucket, " + + "a larger bucket lowers the false negative rate.", + typeConverter=TypeConverters.toFloat) + + @keyword_only + def __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None): + """ + __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None) + """ + super(RandomProjection, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RandomProjection", + self.uid) + self._setDefault(outputDim=1) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + @since("2.1.0") + def setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None): + """ + setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None) + Sets params for this RandomProjection. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + @since("2.1.0") + def setBucketLength(self, value): + """ + Sets the value of :py:attr:`bucketLength`. + """ + return self._set(bucketLength=value) + + @since("2.1.0") + def getBucketLength(self): + """ + Gets the value of bucketLength or its default value. + """ + return self.getOrDefault(self.bucketLength) + + def _create_model(self, java_model): + return RandomProjectionModel(java_model) + + +class RandomProjectionModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): + """ + .. note:: Experimental + + Model fitted by :py:class:`RandomProjection`, where multiple random vectors are stored. + The vectors are normalized to be unit vectors and each vector is used in a hash function: + :math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random unit + vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`. + + .. versionadded:: 2.1.0 + """ + + @property + @since("2.1.0") + def randUnitVectors(self): + """ + An array of random unit vectors. Each vector represents a hash function. + """ + return self._call_java("randUnitVectors") + + @inherit_doc @ignore_unicode_prefix class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): From cdeca1cdd8ed61274137c3012ba49ff57d459190 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 4 Nov 2016 07:44:52 -0700 Subject: [PATCH 02/20] Fix typos. --- python/pyspark/ml/feature.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 37181302d9a2a..5dd7db289263d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -699,7 +699,8 @@ def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbin * Single Probing: Fast, return at most k elements (Probing only one buckets) - * Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to the key) + * Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to \ + the key) :param dataset: The dataset to search for nearest neighbors of the key. :param key: Feature vector representing the item to search for. @@ -824,8 +825,8 @@ class MinHash(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, The input can be dense or sparse vectors, but it is more efficient if it is sparse. For example, `Vectors.sparse(10, Array[(2, 1.0), (3, 1.0), (5, 1.0)])` means there are 10 elements in the space. This set contains elem 2, elem 3 and elem 5. - Also, any input vector must have at least 1 non-zero indices, and all non-zero values are treated - as binary "1" values. + Also, any input vector must have at least 1 non-zero indices, and all non-zero values + are treated as binary "1" values. .. seealso:: `MinHash `_ @@ -1459,7 +1460,8 @@ class RandomProjection(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasS distance space. The output will be vectors of configurable dimension. Hash value in the same dimension is calculated by the same hash function. - .. seealso:: `Stable Distributions `_ + .. seealso:: `Stable Distributions \ + `_ .. seealso:: `Hashing for Similarity Search: A Survey `_ >>> from pyspark.ml.linalg import Vectors From dafc4d120c0606ccd2be892fb2618a1df676ccd3 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 25 Jan 2017 17:23:53 -0800 Subject: [PATCH 03/20] Changes to fix LSH Python API --- python/pyspark/ml/feature.py | 312 +++++++++++++++++------------------ 1 file changed, 153 insertions(+), 159 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9d9d86a9787f6..09767325e957e 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -28,6 +28,7 @@ from pyspark.ml.common import inherit_doc __all__ = ['Binarizer', + 'BucketedRandomProjectionLSH', 'BucketedRandomProjectionLSHModel', 'Bucketizer', 'ChiSqSelector', 'ChiSqSelectorModel', 'CountVectorizer', 'CountVectorizerModel', @@ -37,7 +38,7 @@ 'IDF', 'IDFModel', 'IndexToString', 'MaxAbsScaler', 'MaxAbsScalerModel', - 'MinHash', 'MinHashModel', + 'MinHashLSH', 'MinHashLSHModel', 'MinMaxScaler', 'MinMaxScalerModel', 'NGram', 'Normalizer', @@ -45,7 +46,6 @@ 'PCA', 'PCAModel', 'PolynomialExpansion', 'QuantileDiscretizer', - 'RandomProjection', 'RandomProjectionModel', 'RegexTokenizer', 'RFormula', 'RFormulaModel', 'SQLTransformer', @@ -122,6 +122,123 @@ def getThreshold(self): return self.getOrDefault(self.threshold) + +@inherit_doc +class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, + JavaMLReadable, JavaMLWritable): + """ + .. note:: Experimental + + LSH class for Euclidean distance metrics. + The input is dense or sparse vectors, each of which represents a point in the Euclidean + distance space. The output will be vectors of configurable dimension. Hash value in the + same dimension is calculated by the same hash function. + + .. seealso:: `Stable Distributions \ + `_ + .. seealso:: `Hashing for Similarity Search: A Survey `_ + + >>> from pyspark.ml.linalg import Vectors + >>> data = [(Vectors.dense([-1.0, -1.0 ]),), + ... (Vectors.dense([-1.0, 1.0 ]),), + ... (Vectors.dense([1.0, -1.0 ]),), + ... (Vectors.dense([1.0, 1.0]),)] + >>> df = spark.createDataFrame(data, ["keys"]) + >>> rp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", seed=12345, bucketLength=1.0) + >>> model = rp.fit(df) + >>> model.randUnitVectors + [DenseVector([-0.3041, 0.9527])] + >>> model.transform(df).head() + Row(keys=DenseVector([-1.0, -1.0]), values=DenseVector([-1.0])) + >>> data2 = [(Vectors.dense([2.0, 2.0 ]),), + ... (Vectors.dense([2.0, 3.0 ]),), + ... (Vectors.dense([3.0, 2.0 ]),), + ... (Vectors.dense([3.0, 3.0]),)] + >>> df2 = spark.createDataFrame(data2, ["keys"]) + >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() + [Row(keys=DenseVector([2.0, 2.0]), values=DenseVector([1.0]), distCol=1.0)] + >>> model.approxSimilarityJoin(df, df2, 3.0).select("distCol").head()[0] + 2.236... + >>> rpPath = temp_path + "/rp" + >>> rp.save(rpPath) + >>> rp2 = BucketedRandomProjectionLSH.load(rpPath) + >>> rp2.getBucketLength() == rp.getBucketLength() + True + >>> modelPath = temp_path + "/rp-model" + >>> model.save(modelPath) + >>> model2 = BucketedRandomProjectionLSHModel.load(modelPath) + >>> model2.randUnitVectors == model.randUnitVectors + True + + .. versionadded:: 2.2.0 + """ + + bucketLength = Param(Params._dummy(), "bucketLength", "the length of each hash bucket, " + + "a larger bucket lowers the false negative rate.", + typeConverter=TypeConverters.toFloat) + + @keyword_only + def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None): + """ + __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None) + """ + super(BucketedRandomProjectionLSH, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.BucketedRandomProjectionLSH", + self.uid) + self._setDefault(numHashTables=1) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + @since("2.2.0") + def setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None): + """ + setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None) + Sets params for this BucketedRandomProjectionLSH. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + @since("2.2.0") + def setBucketLength(self, value): + """ + Sets the value of :py:attr:`bucketLength`. + """ + return self._set(bucketLength=value) + + @since("2.2.0") + def getBucketLength(self): + """ + Gets the value of bucketLength or its default value. + """ + return self.getOrDefault(self.bucketLength) + + def _create_model(self, java_model): + return BucketedRandomProjectionLSHModel(java_model) + + +class BucketedRandomProjectionLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): + """ + .. note:: Experimental + + Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are stored. + The vectors are normalized to be unit vectors and each vector is used in a hash function: + :math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random unit + vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`. + + .. versionadded:: 2.2.0 + """ + + @property + @since("2.2.0") + def randUnitVectors(self): + """ + An array of random unit vectors. Each vector represents a hash function. + """ + return self._call_java("randUnitVectors") + + + @inherit_doc class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ @@ -685,27 +802,27 @@ class LSHParams(Params): Mixin for Locality Sensitive Hashing(LSH) algorithm parameters. """ - outputDim = Param(Params._dummy(), "outputDim", "The output dimension, where increasing " + - "dimensionality lowers the false negative rate, and decreasing " + - "dimensionality improves the running performance.", - typeConverter=TypeConverters.toInt) + numHashTables = Param(Params._dummy(), "numHashTables", "number of hash tables, where " + + "increasing number of hash tables lowers the false negative rate, " + + "and decreasing it improves the running performance.", + typeConverter=TypeConverters.toInt) def __init__(self): super(LSHParams, self).__init__() @since("2.1.0") - def setOutputDim(self, value): + def setNumHashTables(self, value): """ - Sets the value of :py:attr:`outputDim`. + Sets the value of :py:attr:`numHashTables`. """ - return self._set(outputDim=value) + return self._set(numHashTables=value) @since("2.1.0") - def getOutputDim(self): + def getNumHashTables(self): """ - Gets the value of outputDim or its default value. + Gets the value of numHashTables or its default value. """ - return self.getOrDefault(self.outputDim) + return self.getOrDefault(self.numHashTables) class LSHModel(): @@ -838,8 +955,8 @@ def maxAbs(self): @inherit_doc -class MinHash(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, - JavaMLReadable, JavaMLWritable): +class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, + JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -860,7 +977,7 @@ class MinHash(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, ... (Vectors.sparse(10, [3, 4], [1.0, 1.0]),), ... (Vectors.sparse(10, [4, 5], [1.0, 1.0]),)] >>> df = spark.createDataFrame(data, ["keys"]) - >>> mh = MinHash(inputCol="keys", outputCol="values", seed=12345) + >>> mh = MinHashLSH(inputCol="keys", outputCol="values", seed=12345) >>> model = mh.fit(df) >>> model.numEntries 20 @@ -879,12 +996,12 @@ class MinHash(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, 0.666... >>> mhPath = temp_path + "/mh" >>> mh.save(mhPath) - >>> mh2 = MinHash.load(mhPath) + >>> mh2 = MinHashLSH.load(mhPath) >>> mh2.getOutputCol() == mh.getOutputCol() True >>> modelPath = temp_path + "/mh-model" >>> model.save(modelPath) - >>> model2 = MinHashModel.load(modelPath) + >>> model2 = MinHashLSHModel.load(modelPath) >>> model2.numEntries == model.numEntries True >>> model2.randCoefficients == model.randCoefficients @@ -894,55 +1011,47 @@ class MinHash(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, """ @keyword_only - def __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1): + def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1): """ - __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1) + __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1) """ - super(MinHash, self).__init__() - self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinHash", self.uid) - self._setDefault(outputDim=1) + super(MinHashLSH, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.MinHashLSH", self.uid) + self._setDefault(numHashTables=1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - @since("2.1.0") - def setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1): + @since("2.2.0") + def setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1): """ - setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1) - Sets params for this MinHash. + setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1) + Sets params for this MinHashLSH. """ kwargs = self.setParams._input_kwargs return self._set(**kwargs) def _create_model(self, java_model): - return MinHashModel(java_model) + return MinHashLSHModel(java_model) -class MinHashModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): +class MinHashLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model produced by :py:class:`MinHash`, where multiple hash functions are stored. - Each hash function is a perfect hash function: - :math:`h_i(x) = (x * k_i \mod prime) \mod numEntries` - where :math:`k_i` is the i-th coefficient, and both `x` and :math:`k_i` - are from :math:`Z_{prime^*}` + Model produced by :py:class:`MinHashLSH`, where where multiple hash functions are stored. Each + hash function is picked from the following family of hash functions, where a_i and b_i are + randomly chosen integers less than prime:`h_i(x) = ((x \cdot a_i + b_i) \mod prime)` + This hash family is approximately min-wise independent according to the reference. - .. seealso:: `Perfect Hash Function `_ + .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear permutations." + Electronic Journal of Combinatorics 7 (2000): R26. - .. versionadded:: 2.1.0 + .. versionadded:: 2.2.0 """ @property - @since("2.1.0") - def numEntries(self): - """ - The number of entries of the hash functions. - """ - return self._call_java("numEntries") - - @property - @since("2.1.0") + @since("2.2.0") def randCoefficients(self): """ An array of random coefficients, each used by one hash function. @@ -1502,121 +1611,6 @@ def _create_model(self, java_model): handleInvalid=self.getHandleInvalid()) -@inherit_doc -class RandomProjection(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, - JavaMLReadable, JavaMLWritable): - """ - .. note:: Experimental - - LSH class for Euclidean distance metrics. - The input is dense or sparse vectors, each of which represents a point in the Euclidean - distance space. The output will be vectors of configurable dimension. Hash value in the - same dimension is calculated by the same hash function. - - .. seealso:: `Stable Distributions \ - `_ - .. seealso:: `Hashing for Similarity Search: A Survey `_ - - >>> from pyspark.ml.linalg import Vectors - >>> data = [(Vectors.dense([-1.0, -1.0 ]),), - ... (Vectors.dense([-1.0, 1.0 ]),), - ... (Vectors.dense([1.0, -1.0 ]),), - ... (Vectors.dense([1.0, 1.0]),)] - >>> df = spark.createDataFrame(data, ["keys"]) - >>> rp = RandomProjection(inputCol="keys", outputCol="values", seed=12345, bucketLength=1.0) - >>> model = rp.fit(df) - >>> model.randUnitVectors - [DenseVector([-0.3041, 0.9527])] - >>> model.transform(df).head() - Row(keys=DenseVector([-1.0, -1.0]), values=DenseVector([-1.0])) - >>> data2 = [(Vectors.dense([2.0, 2.0 ]),), - ... (Vectors.dense([2.0, 3.0 ]),), - ... (Vectors.dense([3.0, 2.0 ]),), - ... (Vectors.dense([3.0, 3.0]),)] - >>> df2 = spark.createDataFrame(data2, ["keys"]) - >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() - [Row(keys=DenseVector([2.0, 2.0]), values=DenseVector([1.0]), distCol=1.0)] - >>> model.approxSimilarityJoin(df, df2, 3.0).select("distCol").head()[0] - 2.236... - >>> rpPath = temp_path + "/rp" - >>> rp.save(rpPath) - >>> rp2 = RandomProjection.load(rpPath) - >>> rp2.getBucketLength() == rp.getBucketLength() - True - >>> modelPath = temp_path + "/rp-model" - >>> model.save(modelPath) - >>> model2 = RandomProjectionModel.load(modelPath) - >>> model2.randUnitVectors == model.randUnitVectors - True - - .. versionadded:: 2.1.0 - """ - - bucketLength = Param(Params._dummy(), "bucketLength", "the length of each hash bucket, " + - "a larger bucket lowers the false negative rate.", - typeConverter=TypeConverters.toFloat) - - @keyword_only - def __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None): - """ - __init__(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None) - """ - super(RandomProjection, self).__init__() - self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RandomProjection", - self.uid) - self._setDefault(outputDim=1) - kwargs = self.__init__._input_kwargs - self.setParams(**kwargs) - - @keyword_only - @since("2.1.0") - def setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None): - """ - setParams(self, inputCol=None, outputCol=None, seed=None, outputDim=1, bucketLength=None) - Sets params for this RandomProjection. - """ - kwargs = self.setParams._input_kwargs - return self._set(**kwargs) - - @since("2.1.0") - def setBucketLength(self, value): - """ - Sets the value of :py:attr:`bucketLength`. - """ - return self._set(bucketLength=value) - - @since("2.1.0") - def getBucketLength(self): - """ - Gets the value of bucketLength or its default value. - """ - return self.getOrDefault(self.bucketLength) - - def _create_model(self, java_model): - return RandomProjectionModel(java_model) - - -class RandomProjectionModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): - """ - .. note:: Experimental - - Model fitted by :py:class:`RandomProjection`, where multiple random vectors are stored. - The vectors are normalized to be unit vectors and each vector is used in a hash function: - :math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random unit - vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`. - - .. versionadded:: 2.1.0 - """ - - @property - @since("2.1.0") - def randUnitVectors(self): - """ - An array of random unit vectors. Each vector represents a hash function. - """ - return self._call_java("randUnitVectors") - - @inherit_doc @ignore_unicode_prefix class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): From 3a21f2666c907d6d520771b4343af7d877d689bb Mon Sep 17 00:00:00 2001 From: Yunni Date: Thu, 26 Jan 2017 02:20:12 -0500 Subject: [PATCH 04/20] Fix examples and class definition --- python/pyspark/ml/feature.py | 194 ++++++++++++++++------------------- 1 file changed, 91 insertions(+), 103 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 09767325e957e..1c6719290ce99 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -122,6 +122,86 @@ def getThreshold(self): return self.getOrDefault(self.threshold) +class LSHParams(Params): + """ + Mixin for Locality Sensitive Hashing(LSH) algorithm parameters. + """ + + numHashTables = Param(Params._dummy(), "numHashTables", "number of hash tables, where " + + "increasing number of hash tables lowers the false negative rate, " + + "and decreasing it improves the running performance.", + typeConverter=TypeConverters.toInt) + + def __init__(self): + super(LSHParams, self).__init__() + + @since("2.2.0") + def setNumHashTables(self, value): + """ + Sets the value of :py:attr:`numHashTables`. + """ + return self._set(numHashTables=value) + + @since("2.2.0") + def getNumHashTables(self): + """ + Gets the value of numHashTables or its default value. + """ + return self.getOrDefault(self.numHashTables) + + +class LSHModel(): + """ + Mixin for Locality Sensitive Hashing(LSH) models. + """ + + @since("2.2.0") + def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbing=True, + distCol="distCol"): + """ + Given a large dataset and an item, approximately find at most k items which have the + closest distance to the item. If the :py:attr:`outputCol` is missing, the method will + transform the data; if the :py:attr:`outputCol` exists, it will use that. This allows + caching of the transformed data when necessary. + + This method implements two ways of fetching k nearest neighbors: + + * Single Probing: Fast, return at most k elements (Probing only one buckets) + + * Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to \ + the key) + + :param dataset: The dataset to search for nearest neighbors of the key. + :param key: Feature vector representing the item to search for. + :param numNearestNeighbors: The maximum number of nearest neighbors. + :param singleProbing: True for using single probing (default); false for multiple probing. + :param distCol: Output column for storing the distance between each result row and the key. + Use "distCol" as default value if it's not specified. + :return: A dataset containing at most k items closest to the key. A distCol is added + to show the distance between each row and the key. + """ + return self._call_java("approxNearestNeighbors", dataset, key, numNearestNeighbors, + singleProbing, distCol) + + @since("2.2.0") + def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol"): + """ + Join two dataset to approximately find all pairs of rows whose distance are smaller than + the threshold. If the :py:attr:`outputCol` is missing, the method will transform the data; + if the :py:attr:`outputCol` exists, it will use that. This allows caching of the + transformed data when necessary. + + :param datasetA: One of the datasets to join. + :param datasetB: Another dataset to join. + :param threshold: The threshold for the distance of row pairs. + :param distCol: Output column for storing the distance between each result row and the key. + Use "distCol" as default value if it's not specified. + :return: A joined dataset containing pairs of rows. The original rows are in columns + "datasetA" and "datasetB", and a distCol is added to show the distance of + each pair. + """ + return self._call_java("approxSimilarityJoin", datasetA, datasetB, threshold, distCol) + @inherit_doc class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, @@ -797,87 +877,6 @@ def idf(self): return self._call_java("idf") -class LSHParams(Params): - """ - Mixin for Locality Sensitive Hashing(LSH) algorithm parameters. - """ - - numHashTables = Param(Params._dummy(), "numHashTables", "number of hash tables, where " + - "increasing number of hash tables lowers the false negative rate, " + - "and decreasing it improves the running performance.", - typeConverter=TypeConverters.toInt) - - def __init__(self): - super(LSHParams, self).__init__() - - @since("2.1.0") - def setNumHashTables(self, value): - """ - Sets the value of :py:attr:`numHashTables`. - """ - return self._set(numHashTables=value) - - @since("2.1.0") - def getNumHashTables(self): - """ - Gets the value of numHashTables or its default value. - """ - return self.getOrDefault(self.numHashTables) - - -class LSHModel(): - """ - Mixin for Locality Sensitive Hashing(LSH) models. - """ - - @since("2.1.0") - def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbing=True, - distCol="distCol"): - """ - Given a large dataset and an item, approximately find at most k items which have the - closest distance to the item. If the :py:attr:`outputCol` is missing, the method will - transform the data; if the :py:attr:`outputCol` exists, it will use that. This allows - caching of the transformed data when necessary. - - This method implements two ways of fetching k nearest neighbors: - - * Single Probing: Fast, return at most k elements (Probing only one buckets) - - * Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to \ - the key) - - :param dataset: The dataset to search for nearest neighbors of the key. - :param key: Feature vector representing the item to search for. - :param numNearestNeighbors: The maximum number of nearest neighbors. - :param singleProbing: True for using single probing (default); false for multiple probing. - :param distCol: Output column for storing the distance between each result row and the key. - Use "distCol" as default value if it's not specified. - :return: A dataset containing at most k items closest to the key. A distCol is added - to show the distance between each row and the key. - """ - return self._call_java("approxNearestNeighbors", dataset, key, numNearestNeighbors, - singleProbing, distCol) - - @since("2.1.0") - def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol"): - """ - Join two dataset to approximately find all pairs of rows whose distance are smaller than - the threshold. If the :py:attr:`outputCol` is missing, the method will transform the data; - if the :py:attr:`outputCol` exists, it will use that. This allows caching of the - transformed data when necessary. - - :param datasetA: One of the datasets to join. - :param datasetB: Another dataset to join. - :param threshold: The threshold for the distance of row pairs. - :param distCol: Output column for storing the distance between each result row and the key. - Use "distCol" as default value if it's not specified. - :return: A joined dataset containing pairs of rows. The original rows are in columns - "datasetA" and "datasetB", and a distCol is added to show the distance of - each pair. - """ - return self._call_java("approxSimilarityJoin", datasetA, datasetB, threshold, distCol) - - @inherit_doc class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ @@ -971,29 +970,22 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, .. seealso:: `MinHash `_ >>> from pyspark.ml.linalg import Vectors - >>> data = [(Vectors.sparse(10, [0, 1], [1.0, 1.0]),), - ... (Vectors.sparse(10, [1, 2], [1.0, 1.0]),), - ... (Vectors.sparse(10, [2, 3], [1.0, 1.0]),), - ... (Vectors.sparse(10, [3, 4], [1.0, 1.0]),), - ... (Vectors.sparse(10, [4, 5], [1.0, 1.0]),)] + >>> data = [(Vectors.sparse(6, [0, 1, 2], [1.0, 1.0, 1.0]),), + ... (Vectors.sparse(6, [2, 3, 4], [1.0, 1.0, 1.0]),), + ... (Vectors.sparse(6, [0, 2, 4], [1.0, 1.0, 1.0]),)] >>> df = spark.createDataFrame(data, ["keys"]) >>> mh = MinHashLSH(inputCol="keys", outputCol="values", seed=12345) >>> model = mh.fit(df) - >>> model.numEntries - 20 - >>> model.randCoefficients - [776966252] >>> model.transform(df).head() - Row(keys=SparseVector(10, {0: 1.0, 1: 1.0}), values=DenseVector([4.0])) - >>> data2 = [(Vectors.sparse(10, [5, 6], [1.0, 1.0]),), - ... (Vectors.sparse(10, [6, 7], [1.0, 1.0]),), - ... (Vectors.sparse(10, [7, 8], [1.0, 1.0]),), - ... (Vectors.sparse(10, [8, 9], [1.0, 1.0]),)] + Row(keys=SparseVector(10, {0: 1.0, 1: 1.0}), values=[DenseVector([-1638925712.0])]) + >>> data2 = [(Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), + ... (Vectors.sparse(6, [2, 3, 5], [1.0, 1.0, 1.0]),), + ... (Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] >>> df2 = spark.createDataFrame(data2, ["keys"]) - >>> model.approxNearestNeighbors(df2, Vectors.sparse(10, [5, 8], [1.0, 1.0]), 1).collect() - [Row(keys=SparseVector(10, {5: 1.0, 6: 1.0}), values=DenseVector([6.0]), distCol=0.666...)] + >>> model.approxNearestNeighbors(df2, Vectors.sparse(6, [1], [1.0]), 1).collect() + [Row(keys=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), values=[DenseVector([-1638925712.0])], distCol=0.666...)] >>> model.approxSimilarityJoin(df, df2, 1.0).select("distCol").head()[0] - 0.666... + 0.5 >>> mhPath = temp_path + "/mh" >>> mh.save(mhPath) >>> mh2 = MinHashLSH.load(mhPath) @@ -1002,12 +994,8 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, >>> modelPath = temp_path + "/mh-model" >>> model.save(modelPath) >>> model2 = MinHashLSHModel.load(modelPath) - >>> model2.numEntries == model.numEntries - True - >>> model2.randCoefficients == model.randCoefficients - True - .. versionadded:: 2.1.0 + .. versionadded:: 2.2.0 """ @keyword_only From 65dab3ec32f423936f2cb310bbfbc312ece8ac54 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 26 Jan 2017 12:19:22 -0800 Subject: [PATCH 05/20] Add python examples and updated the user guide --- docs/ml-features.md | 17 +++++ .../ml/bucketed_random_projection_lsh.py | 75 +++++++++++++++++++ examples/src/main/python/ml/min_hash_lsh.py | 75 +++++++++++++++++++ 3 files changed, 167 insertions(+) create mode 100644 examples/src/main/python/ml/bucketed_random_projection_lsh.py create mode 100644 examples/src/main/python/ml/min_hash_lsh.py diff --git a/docs/ml-features.md b/docs/ml-features.md index 13d97a2290dc3..80accbe2d5301 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1558,6 +1558,15 @@ for more details on the API. {% include_example java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java %} + +
+ +Refer to the [BucketedRandomProjectionLSH Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.BucketedRandomProjectionLSH) +for more details on the API. + +{% include_example python/ml/bucketed_random_projection_lsh.py %} +
+ ### MinHash for Jaccard Distance @@ -1590,4 +1599,12 @@ for more details on the API. {% include_example java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java %} + +
+ +Refer to the [MinHashLSH Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinHashLSH) +for more details on the API. + +{% include_example python/ml/min_hash_lsh.py %} +
diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh.py b/examples/src/main/python/ml/bucketed_random_projection_lsh.py new file mode 100644 index 0000000000000..f2abcc1f439b0 --- /dev/null +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh.py @@ -0,0 +1,75 @@ +# +# 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 +# $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.py +""" + +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", "keys"]) + + 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", "keys"]) + + key = Vectors.dense([1.0, 0.0]) + + brp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", bucketLength=2.0, numHashTables=3) + model = brp.fit(dfA) + + # Feature Transformation + model.transform(dfA).show() + # Cache the transformed columns + transformedA = model.transform(dfA).cache() + transformedB = model.transform(dfB).cache() + + # Approximate similarity join + model.approxSimilarityJoin(dfA, dfB, 1.5).show() + model.approxSimilarityJoin(transformedA, transformedB, 1.5).show() + # Self Join + model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show() + + # Approximate nearest neighbor search + model.approxNearestNeighbors(dfA, key, 2).show() + model.approxNearestNeighbors(transformedA, key, 2).show() + + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/min_hash_lsh.py b/examples/src/main/python/ml/min_hash_lsh.py new file mode 100644 index 0000000000000..755cfe1917e29 --- /dev/null +++ b/examples/src/main/python/ml/min_hash_lsh.py @@ -0,0 +1,75 @@ +# +# 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 +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating MinHashLSH. +Run with: + bin/spark-submit examples/src/main/python/ml/min_hash_lsh.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", "keys"]) + + 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", "keys"]) + + key = Vectors.sparse(6, [1, 3], [1.0, 1.0]) + + mh = MinHashLSH(inputCol="keys", outputCol="values", numHashTables=3) + model = mh.fit(dfA) + + # Feature Transformation + model.transform(dfA).show() + + # Cache the transformed columns + transformedA = model.transform(dfA).cache() + transformedB = model.transform(dfB).cache() + + # Approximate similarity join + model.approxSimilarityJoin(dfA, dfB, 0.6).show() + model.approxSimilarityJoin(transformedA, transformedB, 0.6).show() + + # Self Join + model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show() + + # Approximate nearest neighbor search + model.approxNearestNeighbors(dfA, key, 2).show() + model.approxNearestNeighbors(transformedA, key, 2).show() + + # $example off$ + + spark.stop() From 3d3bcf040831c2a84fbc22f3768800f0b1894c4a Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 26 Jan 2017 13:06:43 -0800 Subject: [PATCH 06/20] Fix lint issues --- .../ml/bucketed_random_projection_lsh.py | 3 +- python/pyspark/ml/feature.py | 34 +++++++++++-------- 2 files changed, 21 insertions(+), 16 deletions(-) diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh.py b/examples/src/main/python/ml/bucketed_random_projection_lsh.py index f2abcc1f439b0..c714948a35bee 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh.py @@ -51,7 +51,8 @@ key = Vectors.dense([1.0, 0.0]) - brp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", bucketLength=2.0, numHashTables=3) + brp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", bucketLength=2.0, + numHashTables=3) model = brp.fit(dfA) # Feature Transformation diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1c6719290ce99..b564dec62209d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -224,7 +224,8 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp ... (Vectors.dense([1.0, -1.0 ]),), ... (Vectors.dense([1.0, 1.0]),)] >>> df = spark.createDataFrame(data, ["keys"]) - >>> rp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", seed=12345, bucketLength=1.0) + >>> rp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", + ... seed=12345, bucketLength=1.0) >>> model = rp.fit(df) >>> model.randUnitVectors [DenseVector([-0.3041, 0.9527])] @@ -258,22 +259,26 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp typeConverter=TypeConverters.toFloat) @keyword_only - def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None): + def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, + bucketLength=None): """ - __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None) + __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, + bucketLength=None) """ super(BucketedRandomProjectionLSH, self).__init__() - self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.BucketedRandomProjectionLSH", - self.uid) + self._java_obj = self._new_java_obj( + "org.apache.spark.ml.feature.BucketedRandomProjectionLSH", self.uid) self._setDefault(numHashTables=1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("2.2.0") - def setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None): + def setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, + bucketLength=None): """ - setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None) + setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, + bucketLength=None) Sets params for this BucketedRandomProjectionLSH. """ kwargs = self.setParams._input_kwargs @@ -301,10 +306,10 @@ class BucketedRandomProjectionLSHModel(JavaModel, LSHModel, JavaMLReadable, Java """ .. note:: Experimental - Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are stored. - The vectors are normalized to be unit vectors and each vector is used in a hash function: - :math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random unit - vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`. + Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are + stored. The vectors are normalized to be unit vectors and each vector is used in a hash + function: :math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random + unit vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`. .. versionadded:: 2.2.0 """ @@ -318,7 +323,6 @@ def randUnitVectors(self): return self._call_java("randUnitVectors") - @inherit_doc class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ @@ -983,7 +987,7 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, ... (Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] >>> df2 = spark.createDataFrame(data2, ["keys"]) >>> model.approxNearestNeighbors(df2, Vectors.sparse(6, [1], [1.0]), 1).collect() - [Row(keys=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), values=[DenseVector([-1638925712.0])], distCol=0.666...)] + [Row(keys=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), values=[DenseVector([-1638925712.0])]...)] >>> model.approxSimilarityJoin(df, df2, 1.0).select("distCol").head()[0] 0.5 >>> mhPath = temp_path + "/mh" @@ -1032,8 +1036,8 @@ class MinHashLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): randomly chosen integers less than prime:`h_i(x) = ((x \cdot a_i + b_i) \mod prime)` This hash family is approximately min-wise independent according to the reference. - .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear permutations." - Electronic Journal of Combinatorics 7 (2000): R26. + .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear + permutations." Electronic Journal of Combinatorics 7 (2000): R26. .. versionadded:: 2.2.0 """ From 69dccdeaab8c36f5433e31a601417f16ba993ccc Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 26 Jan 2017 13:43:52 -0800 Subject: [PATCH 07/20] Fix python doc issues --- python/pyspark/ml/feature.py | 28 +++++++++++----------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index b564dec62209d..d56fa844f8a1c 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -164,24 +164,16 @@ def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbin transform the data; if the :py:attr:`outputCol` exists, it will use that. This allows caching of the transformed data when necessary. - This method implements two ways of fetching k nearest neighbors: - - * Single Probing: Fast, return at most k elements (Probing only one buckets) - - * Multiple Probing: Slow, return exact k elements (Probing multiple buckets close to \ - the key) - :param dataset: The dataset to search for nearest neighbors of the key. :param key: Feature vector representing the item to search for. :param numNearestNeighbors: The maximum number of nearest neighbors. - :param singleProbing: True for using single probing (default); false for multiple probing. :param distCol: Output column for storing the distance between each result row and the key. Use "distCol" as default value if it's not specified. :return: A dataset containing at most k items closest to the key. A distCol is added to show the distance between each row and the key. """ return self._call_java("approxNearestNeighbors", dataset, key, numNearestNeighbors, - singleProbing, distCol) + distCol) @since("2.2.0") def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol"): @@ -266,8 +258,8 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None) """ super(BucketedRandomProjectionLSH, self).__init__() - self._java_obj = self._new_java_obj( - "org.apache.spark.ml.feature.BucketedRandomProjectionLSH", self.uid) + self._java_obj = \ + self._new_java_obj("org.apache.spark.ml.feature.BucketedRandomProjectionLSH", self.uid) self._setDefault(numHashTables=1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -308,8 +300,9 @@ class BucketedRandomProjectionLSHModel(JavaModel, LSHModel, JavaMLReadable, Java Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are stored. The vectors are normalized to be unit vectors and each vector is used in a hash - function: :math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random - unit vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`. + function: :math:`h_i(x) = floor(r_i \cdot x / bucketLength)` where :math:`r_i` is the + i-th random unit vector. The number of buckets will be `(max L2 norm of input vectors) / + bucketLength`. .. versionadded:: 2.2.0 """ @@ -1032,11 +1025,12 @@ class MinHashLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): .. note:: Experimental Model produced by :py:class:`MinHashLSH`, where where multiple hash functions are stored. Each - hash function is picked from the following family of hash functions, where a_i and b_i are - randomly chosen integers less than prime:`h_i(x) = ((x \cdot a_i + b_i) \mod prime)` - This hash family is approximately min-wise independent according to the reference. + hash function is picked from the following family of hash functions, where :math:`a_i` and + :math:`b_i` are randomly chosen integers less than prime: + :math:`h_i(x) = ((x \cdot a_i + b_i) \mod prime)` This hash family is approximately min-wise + independent according to the reference. - .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear + .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear \ permutations." Electronic Journal of Combinatorics 7 (2000): R26. .. versionadded:: 2.2.0 From e7542d0a728c76df0328f69ce4080aec1c87818e Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 26 Jan 2017 14:01:57 -0800 Subject: [PATCH 08/20] Fix 'Definition list ends without a blank line' --- python/pyspark/ml/feature.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index d56fa844f8a1c..7cbb521ab08d6 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -269,7 +269,7 @@ def __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, def setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None): """ - setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, + setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, \ bucketLength=None) Sets params for this BucketedRandomProjectionLSH. """ From 5cfc9c5ff05c413aa7926b7f39aa249941860326 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 26 Jan 2017 15:08:30 -0800 Subject: [PATCH 09/20] Fix python unit tests --- python/pyspark/ml/feature.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 7cbb521ab08d6..1c3ac2ace93b6 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -222,14 +222,14 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp >>> model.randUnitVectors [DenseVector([-0.3041, 0.9527])] >>> model.transform(df).head() - Row(keys=DenseVector([-1.0, -1.0]), values=DenseVector([-1.0])) + Row(keys=DenseVector([-1.0, -1.0]), values=[DenseVector([-1.0])]) >>> data2 = [(Vectors.dense([2.0, 2.0 ]),), ... (Vectors.dense([2.0, 3.0 ]),), ... (Vectors.dense([3.0, 2.0 ]),), ... (Vectors.dense([3.0, 3.0]),)] >>> df2 = spark.createDataFrame(data2, ["keys"]) >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() - [Row(keys=DenseVector([2.0, 2.0]), values=DenseVector([1.0]), distCol=1.0)] + [Row(keys=DenseVector([2.0, 2.0]), values=[DenseVector([1.0])], distCol=1.0)] >>> model.approxSimilarityJoin(df, df2, 3.0).select("distCol").head()[0] 2.236... >>> rpPath = temp_path + "/rp" @@ -974,13 +974,14 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, >>> mh = MinHashLSH(inputCol="keys", outputCol="values", seed=12345) >>> model = mh.fit(df) >>> model.transform(df).head() - Row(keys=SparseVector(10, {0: 1.0, 1: 1.0}), values=[DenseVector([-1638925712.0])]) + Row(keys=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), values=[DenseVector([-1638925712.0])]) >>> data2 = [(Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), ... (Vectors.sparse(6, [2, 3, 5], [1.0, 1.0, 1.0]),), ... (Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] >>> df2 = spark.createDataFrame(data2, ["keys"]) - >>> model.approxNearestNeighbors(df2, Vectors.sparse(6, [1], [1.0]), 1).collect() - [Row(keys=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), values=[DenseVector([-1638925712.0])]...)] + >>> key = Vectors.sparse(6, [1], [1.0]) + >>> model.approxNearestNeighbors(df2, key, 1).select("distCol").head()[0] + 0.66666... >>> model.approxSimilarityJoin(df, df2, 1.0).select("distCol").head()[0] 0.5 >>> mhPath = temp_path + "/mh" From 2508a2fddb058f01e62f50f41380d62bf03f1b56 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 8 Feb 2017 10:56:20 -0800 Subject: [PATCH 10/20] Code Review Comments --- ...bucketed_random_projection_lsh_example.py} | 2 +- ...in_hash_lsh.py => min_hash_lsh_example.py} | 2 +- .../feature/BucketedRandomProjectionLSH.scala | 4 +- .../org/apache/spark/ml/feature/LSH.scala | 2 +- python/pyspark/ml/feature.py | 85 +++++++++---------- 5 files changed, 46 insertions(+), 49 deletions(-) rename examples/src/main/python/ml/{bucketed_random_projection_lsh.py => bucketed_random_projection_lsh_example.py} (99%) rename examples/src/main/python/ml/{min_hash_lsh.py => min_hash_lsh_example.py} (97%) diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py similarity index 99% rename from examples/src/main/python/ml/bucketed_random_projection_lsh.py rename to examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index c714948a35bee..6dc60927234ab 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -27,7 +27,7 @@ """ An example demonstrating BucketedRandomProjectionLSH. Run with: - bin/spark-submit examples/src/main/python/ml/bucketed_random_projection_lsh.py + bin/spark-submit examples/src/main/python/ml/bucketed_random_projection_lsh_example.py """ if __name__ == "__main__": diff --git a/examples/src/main/python/ml/min_hash_lsh.py b/examples/src/main/python/ml/min_hash_lsh_example.py similarity index 97% rename from examples/src/main/python/ml/min_hash_lsh.py rename to examples/src/main/python/ml/min_hash_lsh_example.py index 755cfe1917e29..e6296f2358e66 100644 --- a/examples/src/main/python/ml/min_hash_lsh.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -27,7 +27,7 @@ """ An example demonstrating MinHashLSH. Run with: - bin/spark-submit examples/src/main/python/ml/min_hash_lsh.py + bin/spark-submit examples/src/main/python/ml/min_hash_lsh_example.py """ if __name__ == "__main__": diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala index cbac16345a292..ebb969f99cbf2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala @@ -111,8 +111,8 @@ class BucketedRandomProjectionLSHModel private[ml]( * Euclidean distance metrics. * * The input is dense or sparse vectors, each of which represents a point in the Euclidean - * distance space. The output will be vectors of configurable dimension. Hash values in the - * same dimension are calculated by the same hash function. + * distance space. The output will be vectors of configurable dimension. Hash values in the same + * dimension are calculated by the same hash function. * * References: * diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index 309cc2ef52b04..af452b5f69fd6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -222,7 +222,7 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] } /** - * Join two dataset to approximately find all pairs of rows whose distance are smaller than + * Join two datasets to approximately find all pairs of rows whose distance are smaller than * the threshold. If the [[outputCol]] is missing, the method will transform the data; if the * [[outputCol]] exists, it will use the [[outputCol]]. This allows caching of the transformed * data when necessary. diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1c3ac2ace93b6..ec600a254a3c3 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -124,7 +124,7 @@ def getThreshold(self): class LSHParams(Params): """ - Mixin for Locality Sensitive Hashing(LSH) algorithm parameters. + Mixin for Locality Sensitive Hashing (LSH) algorithm parameters. """ numHashTables = Param(Params._dummy(), "numHashTables", "number of hash tables, where " + @@ -150,20 +150,21 @@ def getNumHashTables(self): return self.getOrDefault(self.numHashTables) -class LSHModel(): +class LSHModel(JavaModel): """ - Mixin for Locality Sensitive Hashing(LSH) models. + Mixin for Locality Sensitive Hashing (LSH) models. """ @since("2.2.0") - def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbing=True, - distCol="distCol"): + def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, distCol="distCol"): """ Given a large dataset and an item, approximately find at most k items which have the closest distance to the item. If the :py:attr:`outputCol` is missing, the method will transform the data; if the :py:attr:`outputCol` exists, it will use that. This allows caching of the transformed data when necessary. + .. note:: This method is experimental and will likely change behavior in the next release. + :param dataset: The dataset to search for nearest neighbors of the key. :param key: Feature vector representing the item to search for. :param numNearestNeighbors: The maximum number of nearest neighbors. @@ -178,7 +179,7 @@ def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, singleProbin @since("2.2.0") def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol"): """ - Join two dataset to approximately find all pairs of rows whose distance are smaller than + Join two datasets to approximately find all pairs of rows whose distance are smaller than the threshold. If the :py:attr:`outputCol` is missing, the method will transform the data; if the :py:attr:`outputCol` exists, it will use that. This allows caching of the transformed data when necessary. @@ -203,8 +204,8 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp LSH class for Euclidean distance metrics. The input is dense or sparse vectors, each of which represents a point in the Euclidean - distance space. The output will be vectors of configurable dimension. Hash value in the - same dimension is calculated by the same hash function. + distance space. The output will be vectors of configurable dimension. Hash values in the same + dimension are calculated by the same hash function. .. seealso:: `Stable Distributions \ `_ @@ -230,18 +231,20 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp >>> df2 = spark.createDataFrame(data2, ["keys"]) >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() [Row(keys=DenseVector([2.0, 2.0]), values=[DenseVector([1.0])], distCol=1.0)] - >>> model.approxSimilarityJoin(df, df2, 3.0).select("distCol").head()[0] - 2.236... - >>> rpPath = temp_path + "/rp" - >>> rp.save(rpPath) - >>> rp2 = BucketedRandomProjectionLSH.load(rpPath) - >>> rp2.getBucketLength() == rp.getBucketLength() + >>> model.approxSimilarityJoin(df, df2, 3.0).show() + +--------------------+--------------------+----------------+ + | datasetA| datasetB| distCol| + +--------------------+--------------------+----------------+ + |[[1.0,1.0],Wrappe...|[[3.0,2.0],Wrappe...|2.23606797749979| + +--------------------+--------------------+----------------+ + >>> brpPath = temp_path + "/brp" + >>> brp.save(rpPath) + >>> brp2 = BucketedRandomProjectionLSH.load(rpPath) + >>> brp2.getBucketLength() == rp.getBucketLength() True - >>> modelPath = temp_path + "/rp-model" + >>> modelPath = temp_path + "/brp-model" >>> model.save(modelPath) >>> model2 = BucketedRandomProjectionLSHModel.load(modelPath) - >>> model2.randUnitVectors == model.randUnitVectors - True .. versionadded:: 2.2.0 """ @@ -294,7 +297,7 @@ def _create_model(self, java_model): return BucketedRandomProjectionLSHModel(java_model) -class BucketedRandomProjectionLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): +class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -307,14 +310,6 @@ class BucketedRandomProjectionLSHModel(JavaModel, LSHModel, JavaMLReadable, Java .. versionadded:: 2.2.0 """ - @property - @since("2.2.0") - def randUnitVectors(self): - """ - An array of random unit vectors. Each vector represents a hash function. - """ - return self._call_java("randUnitVectors") - @inherit_doc class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): @@ -959,12 +954,11 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, LSH class for Jaccard distance. The input can be dense or sparse vectors, but it is more efficient if it is sparse. - For example, `Vectors.sparse(10, Array[(2, 1.0), (3, 1.0), (5, 1.0)])` - means there are 10 elements in the space. This set contains elem 2, elem 3 and elem 5. - Also, any input vector must have at least 1 non-zero indices, and all non-zero values - are treated as binary "1" values. + For example, `Vectors.sparse(10, [(2, 1.0), (3, 1.0), (5, 1.0)])` means there are 10 elements + in the space. This set contains elements 2, 3, and 5. Also, any input vector must have at + least 1 non-zero index, and all non-zero values are treated as binary "1" values. - .. seealso:: `MinHash `_ + .. seealso:: `Wikipedia on MinHash `_ >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.sparse(6, [0, 1, 2], [1.0, 1.0, 1.0]),), @@ -980,10 +974,21 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, ... (Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] >>> df2 = spark.createDataFrame(data2, ["keys"]) >>> key = Vectors.sparse(6, [1], [1.0]) - >>> model.approxNearestNeighbors(df2, key, 1).select("distCol").head()[0] - 0.66666... - >>> model.approxSimilarityJoin(df, df2, 1.0).select("distCol").head()[0] - 0.5 + >>> model.approxNearestNeighbors(df2, key, 1).show() + +--------------------+------------------+------------------+ + | keys| values| distCol| + +--------------------+------------------+------------------+ + |(6,[1,3,5],[1.0,1...|[[-1.638925712E9]]|0.6666666666666667| + +--------------------+------------------+------------------+ + >>> model.approxSimilarityJoin(df, df2, 1.0).show() + +--------------------+--------------------+-------+ + | datasetA| datasetB|distCol| + +--------------------+--------------------+-------+ + |[(6,[2,3,4],[1.0,...|[(6,[2,3,5],[1.0,...| 0.5| + |[(6,[0,2,4],[1.0,...|[(6,[2,3,5],[1.0,...| 0.8| + |[(6,[0,1,2],[1.0,...|[(6,[1,3,5],[1.0,...| 0.8| + |[(6,[0,1,2],[1.0,...|[(6,[1,2,4],[1.0,...| 0.5| + +--------------------+--------------------+-------+ >>> mhPath = temp_path + "/mh" >>> mh.save(mhPath) >>> mh2 = MinHashLSH.load(mhPath) @@ -1021,7 +1026,7 @@ def _create_model(self, java_model): return MinHashLSHModel(java_model) -class MinHashLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): +class MinHashLSHModel(LSHModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -1037,14 +1042,6 @@ class MinHashLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable): .. versionadded:: 2.2.0 """ - @property - @since("2.2.0") - def randCoefficients(self): - """ - An array of random coefficients, each used by one hash function. - """ - return self._call_java("randCoefficients") - @inherit_doc class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): From 8e5468f6946a8f2c051746ddb0c0e65586bd1eed Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 8 Feb 2017 14:38:59 -0800 Subject: [PATCH 11/20] Add printing messages for the LSH Scala/Java/Python exmaples --- ...avaBucketedRandomProjectionLSHExample.java | 7 +++ .../examples/ml/JavaMinHashLSHExample.java | 46 +++++++++++++++++-- .../bucketed_random_projection_lsh_example.py | 7 +++ .../main/python/ml/min_hash_lsh_example.py | 9 +++- .../BucketedRandomProjectionLSHExample.scala | 6 +++ .../spark/examples/ml/MinHashLSHExample.scala | 11 ++++- 6 files changed, 79 insertions(+), 7 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java index ca3ee5a285255..63bc18e29a31b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java @@ -77,19 +77,26 @@ public static void main(String[] args) { BucketedRandomProjectionLSHModel model = mh.fit(dfA); // Feature Transformation + System.out.println("The hashed dataset where hashed values are stored in the column 'values':"); model.transform(dfA).show(); // Cache the transformed columns Dataset transformedA = model.transform(dfA).cache(); Dataset transformedB = model.transform(dfB).cache(); // Approximate similarity join + System.out.println("Approximately joining dfA and dfB on distance smaller than 1.5:"); model.approxSimilarityJoin(dfA, dfB, 1.5).show(); + System.out.println("Joining cached datasets to avoid recomputing the hash values:"); model.approxSimilarityJoin(transformedA, transformedB, 1.5).show(); + // Self Join + System.out.println("Approximately self join of dfB on distance smaller than 2.5:"); model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show(); // Approximate nearest neighbor search + System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); model.approxNearestNeighbors(dfA, key, 2).show(); + System.out.println("Searching cached dataset to avoid recomputing the hash values:"); model.approxNearestNeighbors(transformedA, key, 2).show(); // $example off$ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java index 9dbbf6d117246..aa7b5f9374bfe 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -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$ @@ -44,25 +45,60 @@ public static void main(String[] args) { .getOrCreate(); // $example on$ - List data = Arrays.asList( + List 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 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()) }); - Dataset dataFrame = spark.createDataFrame(data, schema); + Dataset dfA = spark.createDataFrame(dataA, schema); + Dataset dfB = spark.createDataFrame(dataB, schema); + + int[] indicies = {1, 3}; + double[] values = {1.0, 1.0}; + Vector key = Vectors.sparse(6, indicies, values); MinHashLSH mh = new MinHashLSH() - .setNumHashTables(1) + .setNumHashTables(5) .setInputCol("keys") .setOutputCol("values"); - MinHashLSHModel model = mh.fit(dataFrame); - model.transform(dataFrame).show(); + MinHashLSHModel model = mh.fit(dfA); + + // Feature Transformation + System.out.println("The hashed dataset where hashed values are stored in the column 'values':"); + model.transform(dfA).show(); + // Cache the transformed columns + Dataset transformedA = model.transform(dfA).cache(); + Dataset transformedB = model.transform(dfB).cache(); + + // Approximate similarity join + System.out.println("Approximately joining dfA and dfB on distance smaller than 0.6:"); + model.approxSimilarityJoin(dfA, dfB, 0.6).show(); + System.out.println("Joining cached datasets to avoid recomputing the hash values:"); + model.approxSimilarityJoin(transformedA, transformedB, 0.6).show(); + + // Self Join + System.out.println("Approximately self join of dfB on distance smaller than 0.6:"); + model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show(); + + // Approximate nearest neighbor search + System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); + System.out.println("Note: It may return less than 2 rows because of lack of elements in " + + "the hash buckets."); + model.approxNearestNeighbors(dfA, key, 2).show(); + System.out.println("Searching cached dataset to avoid recomputing the hash values:"); + model.approxNearestNeighbors(transformedA, key, 2).show(); // $example off$ spark.stop(); diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index 6dc60927234ab..7211d4381e61a 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -56,19 +56,26 @@ model = brp.fit(dfA) # Feature Transformation + print("The hashed dataset where hashed values are stored in the column 'values':") model.transform(dfA).show() # Cache the transformed columns transformedA = model.transform(dfA).cache() transformedB = model.transform(dfB).cache() # Approximate similarity join + print("Approximately joining dfA and dfB on distance smaller than 1.5:") model.approxSimilarityJoin(dfA, dfB, 1.5).show() + print("Joining cached datasets to avoid recomputing the hash values:") model.approxSimilarityJoin(transformedA, transformedB, 1.5).show() + # Self Join + print("Approximately self join of dfB on distance smaller than 2.5:") model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show() # Approximate nearest neighbor search + print("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() + print("Searching cached dataset to avoid recomputing the hash values:") model.approxNearestNeighbors(transformedA, key, 2).show() # $example off$ diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py index e6296f2358e66..abcfb15261e53 100644 --- a/examples/src/main/python/ml/min_hash_lsh_example.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -49,10 +49,11 @@ key = Vectors.sparse(6, [1, 3], [1.0, 1.0]) - mh = MinHashLSH(inputCol="keys", outputCol="values", numHashTables=3) + mh = MinHashLSH(inputCol="keys", outputCol="values", numHashTables=5) model = mh.fit(dfA) # Feature Transformation + print("The hashed dataset where hashed values are stored in the column 'values':") model.transform(dfA).show() # Cache the transformed columns @@ -60,14 +61,20 @@ transformedB = model.transform(dfB).cache() # Approximate similarity join + print("Approximately joining dfA and dfB on distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfB, 0.6).show() + print("Joining cached datasets to avoid recomputing the hash values:") model.approxSimilarityJoin(transformedA, transformedB, 0.6).show() # Self Join + print("Approximately self join of dfB on distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show() # Approximate nearest neighbor search + print("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() + print("Note: It may return less than 2 rows because of lack of elements in the hash buckets.") + print("Searching cached dataset to avoid recomputing the hash values:") model.approxNearestNeighbors(transformedA, key, 2).show() # $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala index 686cc39d3b9a5..b0e2bb2a6f865 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -58,19 +58,25 @@ object BucketedRandomProjectionLSHExample { val model = brp.fit(dfA) // Feature Transformation + println("The hashed dataset where hashed values are stored in the column 'values':") model.transform(dfA).show() // Cache the transformed columns val transformedA = model.transform(dfA).cache() val transformedB = model.transform(dfB).cache() // Approximate similarity join + println("Approximately joining dfA and dfB on distance smaller than 1.5:") model.approxSimilarityJoin(dfA, dfB, 1.5).show() + println("Joining cached datasets to avoid recomputing the hash values:") model.approxSimilarityJoin(transformedA, transformedB, 1.5).show() // Self Join + println("Approximately self join of dfB on distance smaller than 2.5:") model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show() // Approximate nearest neighbor search + println("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() + println("Searching cached dataset to avoid recomputing the hash values:") model.approxNearestNeighbors(transformedA, key, 2).show() // $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala index f4fc3cf4118a6..9c9bd280087ba 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -48,26 +48,35 @@ object MinHashLSHExample { val key = Vectors.sparse(6, Seq((1, 1.0), (3, 1.0))) val mh = new MinHashLSH() - .setNumHashTables(3) + .setNumHashTables(5) .setInputCol("keys") .setOutputCol("values") val model = mh.fit(dfA) // Feature Transformation + println("The hashed dataset where hashed values are stored in the column 'values':") model.transform(dfA).show() // Cache the transformed columns val transformedA = model.transform(dfA).cache() val transformedB = model.transform(dfB).cache() // Approximate similarity join + println("Approximately joining dfA and dfB on distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfB, 0.6).show() + println("Joining cached datasets to avoid recomputing the hash values:") model.approxSimilarityJoin(transformedA, transformedB, 0.6).show() + // Self Join + println("Approximately self join of dfB on distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show() // Approximate nearest neighbor search + println("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() + println("Note: It may return less than 2 rows because of lack of elements in the hash " + + "buckets.") + println("Searching cached dataset to avoid recomputing the hash values:") model.approxNearestNeighbors(transformedA, key, 2).show() // $example off$ From 6e85e1a04b02dea26e82c8bc77151b8e389f4fe5 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 8 Feb 2017 15:31:53 -0800 Subject: [PATCH 12/20] (1) Rename 'keys''values' to 'features''hashes' (2) Printing the ids for approxSimilarityJoin instead of printing the whole dataset --- ...avaBucketedRandomProjectionLSHExample.java | 6 +- .../examples/ml/JavaMinHashLSHExample.java | 19 ++-- .../bucketed_random_projection_lsh_example.py | 15 +-- .../main/python/ml/min_hash_lsh_example.py | 15 +-- .../BucketedRandomProjectionLSHExample.scala | 22 +++-- .../spark/examples/ml/MinHashLSHExample.scala | 21 +++-- python/pyspark/ml/feature.py | 92 +++++++++---------- 7 files changed, 108 insertions(+), 82 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java index 63bc18e29a31b..4b995abb1ffa2 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java @@ -61,7 +61,7 @@ public static void main(String[] args) { 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 dfA = spark.createDataFrame(dataA, schema); Dataset dfB = spark.createDataFrame(dataB, schema); @@ -71,8 +71,8 @@ public static void main(String[] args) { BucketedRandomProjectionLSH mh = new BucketedRandomProjectionLSH() .setBucketLength(2.0) .setNumHashTables(3) - .setInputCol("keys") - .setOutputCol("values"); + .setInputCol("features") + .setOutputCol("hashes"); BucketedRandomProjectionLSHModel model = mh.fit(dfA); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java index aa7b5f9374bfe..0be9e69d94b58 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -59,7 +59,7 @@ public static void main(String[] args) { 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 dfA = spark.createDataFrame(dataA, schema); Dataset dfB = spark.createDataFrame(dataB, schema); @@ -70,8 +70,8 @@ public static void main(String[] args) { MinHashLSH mh = new MinHashLSH() .setNumHashTables(5) - .setInputCol("keys") - .setOutputCol("values"); + .setInputCol("features") + .setOutputCol("hashes"); MinHashLSHModel model = mh.fit(dfA); @@ -84,13 +84,20 @@ public static void main(String[] args) { // Approximate similarity join System.out.println("Approximately joining dfA and dfB on distance smaller than 0.6:"); - model.approxSimilarityJoin(dfA, dfB, 0.6).show(); + model.approxSimilarityJoin(dfA, dfB, 0.6) + .select("datasetA.id", "datasetB.id", "distCol") + .show(); System.out.println("Joining cached datasets to avoid recomputing the hash values:"); - model.approxSimilarityJoin(transformedA, transformedB, 0.6).show(); + model.approxSimilarityJoin(transformedA, transformedB, 0.6) + .select("datasetA.id", "datasetB.id", "distCol") + .show(); // Self Join System.out.println("Approximately self join of dfB on distance smaller than 0.6:"); - model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show(); + model.approxSimilarityJoin(dfA, dfA, 0.6) + .filter("datasetA.id < datasetB.id") + .select("datasetA.id", "datasetB.id", "distCol") + .show(); // Approximate nearest neighbor search System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index 7211d4381e61a..4b1548a5b82f7 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -41,17 +41,17 @@ (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", "keys"]) + 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", "keys"]) + dfB = spark.createDataFrame(dataB, ["id", "features"]) key = Vectors.dense([1.0, 0.0]) - brp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", bucketLength=2.0, + brp = BucketedRandomProjectionLSH(inputCol="features", outputCol="hashes", bucketLength=2.0, numHashTables=3) model = brp.fit(dfA) @@ -64,13 +64,16 @@ # Approximate similarity join print("Approximately joining dfA and dfB on distance smaller than 1.5:") - model.approxSimilarityJoin(dfA, dfB, 1.5).show() + model.approxSimilarityJoin(dfA, dfB, 1.5)\ + .select("datasetA.id", "datasetB.id", "distCol").show() print("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 1.5).show() + model.approxSimilarityJoin(transformedA, transformedB, 1.5)\ + .select("datasetA.id", "datasetB.id", "distCol").show() # Self Join print("Approximately self join of dfB on distance smaller than 2.5:") - model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show() + model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id")\ + .select("datasetA.id", "datasetB.id", "distCol").show() # Approximate nearest neighbor search print("Approximately searching dfA for 2 nearest neighbors of the key:") diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py index abcfb15261e53..25ec948be66ce 100644 --- a/examples/src/main/python/ml/min_hash_lsh_example.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -40,16 +40,16 @@ 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", "keys"]) + 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", "keys"]) + dfB = spark.createDataFrame(dataB, ["id", "features"]) key = Vectors.sparse(6, [1, 3], [1.0, 1.0]) - mh = MinHashLSH(inputCol="keys", outputCol="values", numHashTables=5) + mh = MinHashLSH(inputCol="features", outputCol="hashes", numHashTables=5) model = mh.fit(dfA) # Feature Transformation @@ -62,13 +62,16 @@ # Approximate similarity join print("Approximately joining dfA and dfB on distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfB, 0.6).show() + model.approxSimilarityJoin(dfA, dfB, 0.6)\ + .select("datasetA.id", "datasetB.id", "distCol").show() print("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 0.6).show() + model.approxSimilarityJoin(transformedA, transformedB, 0.6)\ + .select("datasetA.id", "datasetB.id", "distCol").show() # Self Join print("Approximately self join of dfB on distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show() + model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id")\ + .select("datasetA.id", "datasetB.id", "distCol").show() # Approximate nearest neighbor search print("Approximately searching dfA for 2 nearest neighbors of the key:") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala index b0e2bb2a6f865..62990430f3193 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -38,22 +38,22 @@ object BucketedRandomProjectionLSHExample { (1, Vectors.dense(1.0, -1.0)), (2, Vectors.dense(-1.0, -1.0)), (3, Vectors.dense(-1.0, 1.0)) - )).toDF("id", "keys") + )).toDF("id", "features") val dfB = spark.createDataFrame(Seq( (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)) - )).toDF("id", "keys") + )).toDF("id", "features") val key = Vectors.dense(1.0, 0.0) val brp = new BucketedRandomProjectionLSH() .setBucketLength(2.0) .setNumHashTables(3) - .setInputCol("keys") - .setOutputCol("values") + .setInputCol("features") + .setOutputCol("hashes") val model = brp.fit(dfA) @@ -66,12 +66,20 @@ object BucketedRandomProjectionLSHExample { // Approximate similarity join println("Approximately joining dfA and dfB on distance smaller than 1.5:") - model.approxSimilarityJoin(dfA, dfB, 1.5).show() + model.approxSimilarityJoin(dfA, dfB, 1.5) + .select("datasetA.id", "datasetB.id", "distCol") + .show() println("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 1.5).show() + model.approxSimilarityJoin(transformedA, transformedB, 1.5) + .select("datasetA.id", "datasetB.id", "distCol") + .show() + // Self Join println("Approximately self join of dfB on distance smaller than 2.5:") - model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show() + model.approxSimilarityJoin(dfA, dfA, 2.5) + .filter("datasetA.id < datasetB.id") + .select("datasetA.id", "datasetB.id", "distCol") + .show() // Approximate nearest neighbor search println("Approximately searching dfA for 2 nearest neighbors of the key:") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala index 9c9bd280087ba..4ccfa028e19c7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -37,20 +37,20 @@ object MinHashLSHExample { (0, Vectors.sparse(6, Seq((0, 1.0), (1, 1.0), (2, 1.0)))), (1, Vectors.sparse(6, Seq((2, 1.0), (3, 1.0), (4, 1.0)))), (2, Vectors.sparse(6, Seq((0, 1.0), (2, 1.0), (4, 1.0)))) - )).toDF("id", "keys") + )).toDF("id", "features") val dfB = spark.createDataFrame(Seq( (3, Vectors.sparse(6, Seq((1, 1.0), (3, 1.0), (5, 1.0)))), (4, Vectors.sparse(6, Seq((2, 1.0), (3, 1.0), (5, 1.0)))), (5, Vectors.sparse(6, Seq((1, 1.0), (2, 1.0), (4, 1.0)))) - )).toDF("id", "keys") + )).toDF("id", "features") val key = Vectors.sparse(6, Seq((1, 1.0), (3, 1.0))) val mh = new MinHashLSH() .setNumHashTables(5) - .setInputCol("keys") - .setOutputCol("values") + .setInputCol("features") + .setOutputCol("hashes") val model = mh.fit(dfA) @@ -63,13 +63,20 @@ object MinHashLSHExample { // Approximate similarity join println("Approximately joining dfA and dfB on distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfB, 0.6).show() + model.approxSimilarityJoin(dfA, dfB, 0.6) + .select("datasetA.id", "datasetB.id", "distCol") + .show() println("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 0.6).show() + model.approxSimilarityJoin(transformedA, transformedB, 0.6) + .select("datasetA.id", "datasetB.id", "distCol") + .show() // Self Join println("Approximately self join of dfB on distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id").show() + model.approxSimilarityJoin(dfA, dfA, 0.6) + .filter("datasetA.id < datasetB.id") + .select("datasetA.id", "datasetB.id", "distCol") + .show() // Approximate nearest neighbor search println("Approximately searching dfA for 2 nearest neighbors of the key:") diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index ec600a254a3c3..e3d37a769d9e8 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -212,31 +212,31 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp .. seealso:: `Hashing for Similarity Search: A Survey `_ >>> from pyspark.ml.linalg import Vectors - >>> data = [(Vectors.dense([-1.0, -1.0 ]),), - ... (Vectors.dense([-1.0, 1.0 ]),), - ... (Vectors.dense([1.0, -1.0 ]),), - ... (Vectors.dense([1.0, 1.0]),)] - >>> df = spark.createDataFrame(data, ["keys"]) - >>> rp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", - ... seed=12345, bucketLength=1.0) - >>> model = rp.fit(df) - >>> model.randUnitVectors - [DenseVector([-0.3041, 0.9527])] + >>> data = [(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]),)] + >>> df = spark.createDataFrame(data, ["id", "features"]) + >>> brp = BucketedRandomProjectionLSH(inputCol="features", outputCol="hashes", + ... seed=12345, bucketLength=1.0) + >>> model = brp.fit(df) >>> model.transform(df).head() - Row(keys=DenseVector([-1.0, -1.0]), values=[DenseVector([-1.0])]) - >>> data2 = [(Vectors.dense([2.0, 2.0 ]),), - ... (Vectors.dense([2.0, 3.0 ]),), - ... (Vectors.dense([3.0, 2.0 ]),), - ... (Vectors.dense([3.0, 3.0]),)] - >>> df2 = spark.createDataFrame(data2, ["keys"]) + Row(id=0, features=DenseVector([-1.0, -1.0]), hashes=[DenseVector([-1.0])]) + >>> data2 = [(4, Vectors.dense([2.0, 2.0 ]),), + ... (5, Vectors.dense([2.0, 3.0 ]),), + ... (6, Vectors.dense([3.0, 2.0 ]),), + ... (7, Vectors.dense([3.0, 3.0]),)] + >>> df2 = spark.createDataFrame(data2, ["id", "features"]) >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() - [Row(keys=DenseVector([2.0, 2.0]), values=[DenseVector([1.0])], distCol=1.0)] - >>> model.approxSimilarityJoin(df, df2, 3.0).show() - +--------------------+--------------------+----------------+ - | datasetA| datasetB| distCol| - +--------------------+--------------------+----------------+ - |[[1.0,1.0],Wrappe...|[[3.0,2.0],Wrappe...|2.23606797749979| - +--------------------+--------------------+----------------+ + [Row(id=4, features=DenseVector([2.0, 2.0]), hashes=[DenseVector([1.0])], distCol=1.0)] + >>> model.approxSimilarityJoin(df, df2, 3.0).select("datasetA.id", + ... "datasetB.id", + ... "distCol").show() + +---+---+----------------+ + | id| id| distCol| + +---+---+----------------+ + | 3| 6|2.23606797749979| + +---+---+----------------+ >>> brpPath = temp_path + "/brp" >>> brp.save(rpPath) >>> brp2 = BucketedRandomProjectionLSH.load(rpPath) @@ -961,34 +961,32 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, .. seealso:: `Wikipedia on MinHash `_ >>> from pyspark.ml.linalg import Vectors - >>> data = [(Vectors.sparse(6, [0, 1, 2], [1.0, 1.0, 1.0]),), - ... (Vectors.sparse(6, [2, 3, 4], [1.0, 1.0, 1.0]),), - ... (Vectors.sparse(6, [0, 2, 4], [1.0, 1.0, 1.0]),)] - >>> df = spark.createDataFrame(data, ["keys"]) - >>> mh = MinHashLSH(inputCol="keys", outputCol="values", seed=12345) + >>> data = [(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]),)] + >>> df = spark.createDataFrame(data, ["id", "features"]) + >>> mh = MinHashLSH(inputCol="features", outputCol="hashes", seed=12345) >>> model = mh.fit(df) >>> model.transform(df).head() - Row(keys=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), values=[DenseVector([-1638925712.0])]) - >>> data2 = [(Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), - ... (Vectors.sparse(6, [2, 3, 5], [1.0, 1.0, 1.0]),), - ... (Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] - >>> df2 = spark.createDataFrame(data2, ["keys"]) + Row(id=0, features=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), hashes=[DenseVector([-1638925712.0])]) + >>> data2 = [(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]),)] + >>> df2 = spark.createDataFrame(data2, ["id", "features"]) >>> key = Vectors.sparse(6, [1], [1.0]) >>> model.approxNearestNeighbors(df2, key, 1).show() - +--------------------+------------------+------------------+ - | keys| values| distCol| - +--------------------+------------------+------------------+ - |(6,[1,3,5],[1.0,1...|[[-1.638925712E9]]|0.6666666666666667| - +--------------------+------------------+------------------+ - >>> model.approxSimilarityJoin(df, df2, 1.0).show() - +--------------------+--------------------+-------+ - | datasetA| datasetB|distCol| - +--------------------+--------------------+-------+ - |[(6,[2,3,4],[1.0,...|[(6,[2,3,5],[1.0,...| 0.5| - |[(6,[0,2,4],[1.0,...|[(6,[2,3,5],[1.0,...| 0.8| - |[(6,[0,1,2],[1.0,...|[(6,[1,3,5],[1.0,...| 0.8| - |[(6,[0,1,2],[1.0,...|[(6,[1,2,4],[1.0,...| 0.5| - +--------------------+--------------------+-------+ + +---+--------------------+------------------+------------------+ + | id| features| hashes| distCol| + +---+--------------------+------------------+------------------+ + | 5|(6,[1,2,4],[1.0,1...|[[-1.638925712E9]]|0.6666666666666667| + +---+--------------------+------------------+------------------+ + >>> model.approxSimilarityJoin(df, df2, 0.6).select("datasetA.id", "datasetB.id", "distCol").show() + +---+---+-------+ + | id| id|distCol| + +---+---+-------+ + | 1| 4| 0.5| + | 0| 5| 0.5| + +---+---+-------+ >>> mhPath = temp_path + "/mh" >>> mh.save(mhPath) >>> mh2 = MinHashLSH.load(mhPath) From 4bc670cf4e512953019d97cfd57413158f31377a Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 8 Feb 2017 17:01:49 -0800 Subject: [PATCH 13/20] Fix jenkins build --- python/pyspark/ml/feature.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index e3d37a769d9e8..4c88cb66727f7 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -237,10 +237,11 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp +---+---+----------------+ | 3| 6|2.23606797749979| +---+---+----------------+ + ... >>> brpPath = temp_path + "/brp" - >>> brp.save(rpPath) - >>> brp2 = BucketedRandomProjectionLSH.load(rpPath) - >>> brp2.getBucketLength() == rp.getBucketLength() + >>> brp.save(brpPath) + >>> brp2 = BucketedRandomProjectionLSH.load(brpPath) + >>> brp2.getBucketLength() == brp.getBucketLength() True >>> modelPath = temp_path + "/brp-model" >>> model.save(modelPath) @@ -968,7 +969,7 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, >>> mh = MinHashLSH(inputCol="features", outputCol="hashes", seed=12345) >>> model = mh.fit(df) >>> model.transform(df).head() - Row(id=0, features=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), hashes=[DenseVector([-1638925712.0])]) + Row(id=0, features=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), hashes=[DenseVector([-1638925... >>> data2 = [(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]),)] @@ -980,13 +981,17 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, +---+--------------------+------------------+------------------+ | 5|(6,[1,2,4],[1.0,1...|[[-1.638925712E9]]|0.6666666666666667| +---+--------------------+------------------+------------------+ - >>> model.approxSimilarityJoin(df, df2, 0.6).select("datasetA.id", "datasetB.id", "distCol").show() + ... + >>> model.approxSimilarityJoin(df, df2, 0.6).select("datasetA.id", + ... "datasetB.id", + ... "distCol").show() +---+---+-------+ | id| id|distCol| +---+---+-------+ | 1| 4| 0.5| | 0| 5| 0.5| +---+---+-------+ + ... >>> mhPath = temp_path + "/mh" >>> mh.save(mhPath) >>> mh2 = MinHashLSH.load(mhPath) From b45ec0ab118545383526ffa80fa873a4ccc33307 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 8 Feb 2017 20:17:49 -0800 Subject: [PATCH 14/20] Fix failed jenkins test --- python/pyspark/ml/feature.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 4c88cb66727f7..ce39c4ac4ada2 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -975,13 +975,8 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, ... (5, Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] >>> df2 = spark.createDataFrame(data2, ["id", "features"]) >>> key = Vectors.sparse(6, [1], [1.0]) - >>> model.approxNearestNeighbors(df2, key, 1).show() - +---+--------------------+------------------+------------------+ - | id| features| hashes| distCol| - +---+--------------------+------------------+------------------+ - | 5|(6,[1,2,4],[1.0,1...|[[-1.638925712E9]]|0.6666666666666667| - +---+--------------------+------------------+------------------+ - ... + >>> model.approxNearestNeighbors(df2, key, 1).collect() + [Row(id=5, features=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), hashes=[DenseVector([-163892... >>> model.approxSimilarityJoin(df, df2, 0.6).select("datasetA.id", ... "datasetB.id", ... "distCol").show() From 1b70b919edea26321f21220f11d520d4f4f98ede Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Wed, 8 Feb 2017 22:28:21 -0800 Subject: [PATCH 15/20] Fix Jenkins test --- python/pyspark/ml/feature.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index ce39c4ac4ada2..f9474bf923895 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -974,7 +974,7 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, ... (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]),)] >>> df2 = spark.createDataFrame(data2, ["id", "features"]) - >>> key = Vectors.sparse(6, [1], [1.0]) + >>> key = Vectors.sparse(6, [1, 2], [1.0, 1.0]) >>> model.approxNearestNeighbors(df2, key, 1).collect() [Row(id=5, features=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), hashes=[DenseVector([-163892... >>> model.approxSimilarityJoin(df, df2, 0.6).select("datasetA.id", From b1da01e8a05472d2de8e025dbebdf1fe011a9c17 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 9 Feb 2017 16:12:24 -0800 Subject: [PATCH 16/20] Code Review Comments for the LSH examples --- ...avaBucketedRandomProjectionLSHExample.java | 23 +++++------- .../examples/ml/JavaMinHashLSHExample.java | 37 +++++++------------ .../bucketed_random_projection_lsh_example.py | 28 +++++--------- .../main/python/ml/min_hash_lsh_example.py | 28 +++++--------- .../BucketedRandomProjectionLSHExample.scala | 30 +++++---------- .../spark/examples/ml/MinHashLSHExample.scala | 33 ++++++----------- 6 files changed, 63 insertions(+), 116 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java index 4b995abb1ffa2..c2d0c51b843cd 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java @@ -77,27 +77,22 @@ public static void main(String[] args) { BucketedRandomProjectionLSHModel model = mh.fit(dfA); // Feature Transformation - System.out.println("The hashed dataset where hashed values are stored in the column 'values':"); + System.out.println("The hashed dataset where hashed values are stored in the column 'hashes':"); model.transform(dfA).show(); - // Cache the transformed columns - Dataset transformedA = model.transform(dfA).cache(); - Dataset transformedB = model.transform(dfB).cache(); - // Approximate similarity join + // 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)` System.out.println("Approximately joining dfA and dfB on distance smaller than 1.5:"); model.approxSimilarityJoin(dfA, dfB, 1.5).show(); - System.out.println("Joining cached datasets to avoid recomputing the hash values:"); - model.approxSimilarityJoin(transformedA, transformedB, 1.5).show(); - // Self Join - System.out.println("Approximately self join of dfB on distance smaller than 2.5:"); - model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id").show(); - - // Approximate nearest neighbor search + // 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)` System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); model.approxNearestNeighbors(dfA, key, 2).show(); - System.out.println("Searching cached dataset to avoid recomputing the hash values:"); - model.approxNearestNeighbors(transformedA, key, 2).show(); // $example off$ spark.stop(); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java index 0be9e69d94b58..209461ef91864 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -64,9 +64,9 @@ public static void main(String[] args) { Dataset dfA = spark.createDataFrame(dataA, schema); Dataset dfB = spark.createDataFrame(dataB, schema); - int[] indicies = {1, 3}; + int[] indices = {1, 3}; double[] values = {1.0, 1.0}; - Vector key = Vectors.sparse(6, indicies, values); + Vector key = Vectors.sparse(6, indices, values); MinHashLSH mh = new MinHashLSH() .setNumHashTables(5) @@ -76,36 +76,25 @@ public static void main(String[] args) { MinHashLSHModel model = mh.fit(dfA); // Feature Transformation - System.out.println("The hashed dataset where hashed values are stored in the column 'values':"); + System.out.println("The hashed dataset where hashed values are stored in the column 'hashes':"); model.transform(dfA).show(); - // Cache the transformed columns - Dataset transformedA = model.transform(dfA).cache(); - Dataset transformedB = model.transform(dfB).cache(); - // Approximate similarity join - System.out.println("Approximately joining dfA and dfB on distance smaller than 0.6:"); + // 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) .select("datasetA.id", "datasetB.id", "distCol") .show(); - System.out.println("Joining cached datasets to avoid recomputing the hash values:"); - model.approxSimilarityJoin(transformedA, transformedB, 0.6) - .select("datasetA.id", "datasetB.id", "distCol") - .show(); - - // Self Join - System.out.println("Approximately self join of dfB on distance smaller than 0.6:"); - model.approxSimilarityJoin(dfA, dfA, 0.6) - .filter("datasetA.id < datasetB.id") - .select("datasetA.id", "datasetB.id", "distCol") - .show(); - // Approximate nearest neighbor search + // 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 because of lack of elements in the hash buckets. System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); - System.out.println("Note: It may return less than 2 rows because of lack of elements in " + - "the hash buckets."); model.approxNearestNeighbors(dfA, key, 2).show(); - System.out.println("Searching cached dataset to avoid recomputing the hash values:"); - model.approxNearestNeighbors(transformedA, key, 2).show(); // $example off$ spark.stop(); diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index 4b1548a5b82f7..2907385c6f819 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -56,31 +56,23 @@ model = brp.fit(dfA) # Feature Transformation - print("The hashed dataset where hashed values are stored in the column 'values':") + print("The hashed dataset where hashed values are stored in the column 'hashes':") model.transform(dfA).show() - # Cache the transformed columns - transformedA = model.transform(dfA).cache() - transformedB = model.transform(dfB).cache() - # Approximate similarity join - print("Approximately joining dfA and dfB on distance smaller than 1.5:") + # 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)\ .select("datasetA.id", "datasetB.id", "distCol").show() - print("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 1.5)\ - .select("datasetA.id", "datasetB.id", "distCol").show() - - # Self Join - print("Approximately self join of dfB on distance smaller than 2.5:") - model.approxSimilarityJoin(dfA, dfA, 2.5).filter("datasetA.id < datasetB.id")\ - .select("datasetA.id", "datasetB.id", "distCol").show() - # Approximate nearest neighbor search + # 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() - print("Searching cached dataset to avoid recomputing the hash values:") - model.approxNearestNeighbors(transformedA, key, 2).show() - # $example off$ spark.stop() diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py index 25ec948be66ce..5cd049bc3a9c7 100644 --- a/examples/src/main/python/ml/min_hash_lsh_example.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -53,32 +53,24 @@ model = mh.fit(dfA) # Feature Transformation - print("The hashed dataset where hashed values are stored in the column 'values':") + print("The hashed dataset where hashed values are stored in the column 'hashes':") model.transform(dfA).show() - # Cache the transformed columns - transformedA = model.transform(dfA).cache() - transformedB = model.transform(dfB).cache() - - # Approximate similarity join + # 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)\ .select("datasetA.id", "datasetB.id", "distCol").show() - print("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 0.6)\ - .select("datasetA.id", "datasetB.id", "distCol").show() - - # Self Join - print("Approximately self join of dfB on distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfA, 0.6).filter("datasetA.id < datasetB.id")\ - .select("datasetA.id", "datasetB.id", "distCol").show() - # Approximate nearest neighbor search + # 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 because of lack of elements in the hash buckets. print("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() - print("Note: It may return less than 2 rows because of lack of elements in the hash buckets.") - print("Searching cached dataset to avoid recomputing the hash values:") - model.approxNearestNeighbors(transformedA, key, 2).show() # $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala index 62990430f3193..f0762cb6434e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -58,34 +58,24 @@ object BucketedRandomProjectionLSHExample { val model = brp.fit(dfA) // Feature Transformation - println("The hashed dataset where hashed values are stored in the column 'values':") + println("The hashed dataset where hashed values are stored in the column 'hashes':") model.transform(dfA).show() - // Cache the transformed columns - val transformedA = model.transform(dfA).cache() - val transformedB = model.transform(dfB).cache() - // Approximate similarity join - println("Approximately joining dfA and dfB on distance smaller than 1.5:") + // 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)` + println("Approximately joining dfA and dfB on Euclidean distance smaller than 1.5:") model.approxSimilarityJoin(dfA, dfB, 1.5) .select("datasetA.id", "datasetB.id", "distCol") .show() - println("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 1.5) - .select("datasetA.id", "datasetB.id", "distCol") - .show() - - // Self Join - println("Approximately self join of dfB on distance smaller than 2.5:") - model.approxSimilarityJoin(dfA, dfA, 2.5) - .filter("datasetA.id < datasetB.id") - .select("datasetA.id", "datasetB.id", "distCol") - .show() - // Approximate nearest neighbor search + // 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)` println("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() - println("Searching cached dataset to avoid recomputing the hash values:") - model.approxNearestNeighbors(transformedA, key, 2).show() // $example off$ spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala index 4ccfa028e19c7..27258011d6ee6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -55,36 +55,25 @@ object MinHashLSHExample { val model = mh.fit(dfA) // Feature Transformation - println("The hashed dataset where hashed values are stored in the column 'values':") + println("The hashed dataset where hashed values are stored in the column 'hashes':") model.transform(dfA).show() - // Cache the transformed columns - val transformedA = model.transform(dfA).cache() - val transformedB = model.transform(dfB).cache() - // Approximate similarity join - println("Approximately joining dfA and dfB on distance smaller than 0.6:") + // 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)` + println("Approximately joining dfA and dfB on Jaccard distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfB, 0.6) .select("datasetA.id", "datasetB.id", "distCol") .show() - println("Joining cached datasets to avoid recomputing the hash values:") - model.approxSimilarityJoin(transformedA, transformedB, 0.6) - .select("datasetA.id", "datasetB.id", "distCol") - .show() - - // Self Join - println("Approximately self join of dfB on distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfA, 0.6) - .filter("datasetA.id < datasetB.id") - .select("datasetA.id", "datasetB.id", "distCol") - .show() - // Approximate nearest neighbor search + // 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 because of lack of elements in the hash buckets. println("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() - println("Note: It may return less than 2 rows because of lack of elements in the hash " + - "buckets.") - println("Searching cached dataset to avoid recomputing the hash values:") - model.approxNearestNeighbors(transformedA, key, 2).show() // $example off$ spark.stop() From 8f1d70819e9ed6d8b8bb8540dcc1ca9747b67cae Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Thu, 9 Feb 2017 17:20:51 -0800 Subject: [PATCH 17/20] Add alias for similarity join examples --- ...avaBucketedRandomProjectionLSHExample.java | 7 +++- .../examples/ml/JavaMinHashLSHExample.java | 7 +++- .../bucketed_random_projection_lsh_example.py | 5 ++- .../main/python/ml/min_hash_lsh_example.py | 5 ++- .../BucketedRandomProjectionLSHExample.scala | 6 ++- .../spark/examples/ml/MinHashLSHExample.scala | 6 ++- python/pyspark/ml/feature.py | 38 ++++++++++--------- 7 files changed, 48 insertions(+), 26 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java index c2d0c51b843cd..be0144ba719e3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java @@ -35,6 +35,8 @@ 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.*; // $example off$ public class JavaBucketedRandomProjectionLSHExample { @@ -85,7 +87,10 @@ public static void main(String[] args) { // We could avoid computing hashes by passing in the already-transformed dataset, e.g. // `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` System.out.println("Approximately joining dfA and dfB on distance smaller than 1.5:"); - model.approxSimilarityJoin(dfA, dfB, 1.5).show(); + model.approxSimilarityJoin(dfA, dfB, 1.5) + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("distCol").alias("EuclideanDistance")).show(); // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java index 209461ef91864..ced4c9f2bafb5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -35,6 +35,8 @@ 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.*; // $example off$ public class JavaMinHashLSHExample { @@ -85,8 +87,9 @@ public static void main(String[] args) { // `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) - .select("datasetA.id", "datasetB.id", "distCol") - .show(); + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("distCol").alias("JaccardDistance")).show(); // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index 2907385c6f819..cae190607478a 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -21,6 +21,7 @@ # $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 @@ -65,7 +66,9 @@ # `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)\ - .select("datasetA.id", "datasetB.id", "distCol").show() + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("distCol").alias("EuclideanDistance")).show() # Compute the locality sensitive hashes for the input rows, then perform approximate nearest # neighbor search. diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py index 5cd049bc3a9c7..50e6917a57b5f 100644 --- a/examples/src/main/python/ml/min_hash_lsh_example.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -21,6 +21,7 @@ # $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 @@ -62,7 +63,9 @@ # `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` print("Approximately joining dfA and dfB on distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfB, 0.6)\ - .select("datasetA.id", "datasetB.id", "distCol").show() + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("distCol").alias("JaccardDistance")).show() # Compute the locality sensitive hashes for the input rows, then perform approximate nearest # neighbor search. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala index f0762cb6434e5..c0ac9654413f2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -21,6 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.BucketedRandomProjectionLSH import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.sql.functions._ // $example off$ import org.apache.spark.sql.SparkSession @@ -67,8 +68,9 @@ object BucketedRandomProjectionLSHExample { // `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` println("Approximately joining dfA and dfB on Euclidean distance smaller than 1.5:") model.approxSimilarityJoin(dfA, dfB, 1.5) - .select("datasetA.id", "datasetB.id", "distCol") - .show() + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("distCol").alias("EuclideanDistance")).show() // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala index 27258011d6ee6..181c3922bb02b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -21,6 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.MinHashLSH import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.sql.functions._ // $example off$ import org.apache.spark.sql.SparkSession @@ -64,8 +65,9 @@ object MinHashLSHExample { // `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` println("Approximately joining dfA and dfB on Jaccard distance smaller than 0.6:") model.approxSimilarityJoin(dfA, dfB, 0.6) - .select("datasetA.id", "datasetB.id", "distCol") - .show() + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("distCol").alias("JaccardDistance")).show() // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index f9474bf923895..9833d35f7d2b1 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -212,6 +212,7 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp .. seealso:: `Hashing for Similarity Search: A Survey `_ >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.sql.functions import col >>> data = [(0, Vectors.dense([-1.0, -1.0 ]),), ... (1, Vectors.dense([-1.0, 1.0 ]),), ... (2, Vectors.dense([1.0, -1.0 ]),), @@ -229,14 +230,15 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp >>> df2 = spark.createDataFrame(data2, ["id", "features"]) >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() [Row(id=4, features=DenseVector([2.0, 2.0]), hashes=[DenseVector([1.0])], distCol=1.0)] - >>> model.approxSimilarityJoin(df, df2, 3.0).select("datasetA.id", - ... "datasetB.id", - ... "distCol").show() - +---+---+----------------+ - | id| id| distCol| - +---+---+----------------+ - | 3| 6|2.23606797749979| - +---+---+----------------+ + >>> model.approxSimilarityJoin(df, df2, 3.0).select( + ... col("datasetA.id").alias("idA"), + ... col("datasetB.id").alias("idB"), + ... col("distCol").alias("EuclideanDistance")).show() + +---+---+-----------------+ + |idA|idB|EuclideanDistance| + +---+---+-----------------+ + | 3| 6| 2.23606797749979| + +---+---+-----------------+ ... >>> brpPath = temp_path + "/brp" >>> brp.save(brpPath) @@ -962,6 +964,7 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, .. seealso:: `Wikipedia on MinHash `_ >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.sql.functions import col >>> data = [(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]),)] @@ -977,15 +980,16 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, >>> key = Vectors.sparse(6, [1, 2], [1.0, 1.0]) >>> model.approxNearestNeighbors(df2, key, 1).collect() [Row(id=5, features=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), hashes=[DenseVector([-163892... - >>> model.approxSimilarityJoin(df, df2, 0.6).select("datasetA.id", - ... "datasetB.id", - ... "distCol").show() - +---+---+-------+ - | id| id|distCol| - +---+---+-------+ - | 1| 4| 0.5| - | 0| 5| 0.5| - +---+---+-------+ + >>> model.approxSimilarityJoin(df, df2, 0.6).select( + ... col("datasetA.id").alias("idA"), + ... col("datasetB.id").alias("idB"), + ... col("distCol").alias("JaccardDistance")).show() + +---+---+---------------+ + |idA|idB|JaccardDistance| + +---+---+---------------+ + | 1| 4| 0.5| + | 0| 5| 0.5| + +---+---+---------------+ ... >>> mhPath = temp_path + "/mh" >>> mh.save(mhPath) From c64d50bd5a11f0f284e0964dcfce5a9040d1be99 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Mon, 13 Feb 2017 22:19:00 -0800 Subject: [PATCH 18/20] Code Review Comments --- docs/ml-features.md | 4 ++-- ...avaBucketedRandomProjectionLSHExample.java | 11 ++++++--- .../examples/ml/JavaMinHashLSHExample.java | 14 +++++++---- .../bucketed_random_projection_lsh_example.py | 4 ++-- .../main/python/ml/min_hash_lsh_example.py | 7 +++--- .../BucketedRandomProjectionLSHExample.scala | 9 +++++-- .../spark/examples/ml/MinHashLSHExample.scala | 8 ++++++- .../feature/BucketedRandomProjectionLSH.scala | 4 ++-- .../org/apache/spark/ml/feature/LSH.scala | 5 ++-- python/pyspark/ml/feature.py | 24 ++++++++----------- 10 files changed, 55 insertions(+), 35 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index 80accbe2d5301..57605bafbf4c3 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1564,7 +1564,7 @@ for more details on the API. Refer to the [BucketedRandomProjectionLSH Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.BucketedRandomProjectionLSH) for more details on the API. -{% include_example python/ml/bucketed_random_projection_lsh.py %} +{% include_example python/ml/bucketed_random_projection_lsh_example.py %} @@ -1605,6 +1605,6 @@ for more details on the API. Refer to the [MinHashLSH Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinHashLSH) for more details on the API. -{% include_example python/ml/min_hash_lsh.py %} +{% include_example python/ml/min_hash_lsh_example.py %} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java index be0144ba719e3..4594e3462b2a5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java @@ -36,9 +36,14 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.functions.*; +import static org.apache.spark.sql.functions.col; // $example off$ +/** + * An example demonstrating BucketedRandomProjectionLSH. + * Run with: + * bin/run-example org.apache.spark.examples.ml.JavaBucketedRandomProjectionLSHExample + */ public class JavaBucketedRandomProjectionLSHExample { public static void main(String[] args) { SparkSession spark = SparkSession @@ -87,10 +92,10 @@ public static void main(String[] args) { // We could avoid computing hashes by passing in the already-transformed dataset, e.g. // `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` System.out.println("Approximately joining dfA and dfB on distance smaller than 1.5:"); - model.approxSimilarityJoin(dfA, dfB, 1.5) + model.approxSimilarityJoin(dfA, dfB, 1.5, "EuclideanDistance") .select(col("datasetA.id").alias("idA"), col("datasetB.id").alias("idB"), - col("distCol").alias("EuclideanDistance")).show(); + col("EuclideanDistance")).show(); // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java index ced4c9f2bafb5..e36e5ba1e3445 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -36,9 +36,14 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.functions.*; +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 @@ -86,16 +91,17 @@ public static void main(String[] args) { // 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) + model.approxSimilarityJoin(dfA, dfB, 0.6, "JaccardDistance") .select(col("datasetA.id").alias("idA"), col("datasetB.id").alias("idB"), - col("distCol").alias("JaccardDistance")).show(); + 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 because of lack of elements in the hash buckets. + // 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$ diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py index cae190607478a..1b7a458125cef 100644 --- a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -65,10 +65,10 @@ # 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)\ + model.approxSimilarityJoin(dfA, dfB, 1.5, distCol="EuclideanDistance")\ .select(col("datasetA.id").alias("idA"), col("datasetB.id").alias("idB"), - col("distCol").alias("EuclideanDistance")).show() + col("EuclideanDistance")).show() # Compute the locality sensitive hashes for the input rows, then perform approximate nearest # neighbor search. diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py index 50e6917a57b5f..7b1dd611a865b 100644 --- a/examples/src/main/python/ml/min_hash_lsh_example.py +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -62,16 +62,17 @@ # 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)\ + model.approxSimilarityJoin(dfA, dfB, 0.6, distCol="JaccardDistance")\ .select(col("datasetA.id").alias("idA"), col("datasetB.id").alias("idB"), - col("distCol").alias("JaccardDistance")).show() + 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 because of lack of elements in the hash buckets. + # 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() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala index c0ac9654413f2..cf80659c24246 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -25,6 +25,11 @@ import org.apache.spark.sql.functions._ // $example off$ import org.apache.spark.sql.SparkSession +/** + * An example demonstrating BucketedRandomProjectionLSH. + * Run with: + * bin/run-example org.apache.spark.examples.ml.BucketedRandomProjectionLSHExample + */ object BucketedRandomProjectionLSHExample { def main(args: Array[String]): Unit = { // Creates a SparkSession @@ -67,10 +72,10 @@ object BucketedRandomProjectionLSHExample { // We could avoid computing hashes by passing in the already-transformed dataset, e.g. // `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` println("Approximately joining dfA and dfB on Euclidean distance smaller than 1.5:") - model.approxSimilarityJoin(dfA, dfB, 1.5) + model.approxSimilarityJoin(dfA, dfB, 1.5, "EuclideanDistance") .select(col("datasetA.id").alias("idA"), col("datasetB.id").alias("idB"), - col("distCol").alias("EuclideanDistance")).show() + col("EuclideanDistance")).show() // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala index 181c3922bb02b..92a08e778b6b0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -25,6 +25,11 @@ import org.apache.spark.sql.functions._ // $example off$ import org.apache.spark.sql.SparkSession +/** + * An example demonstrating MinHashLSH. + * Run with: + * bin/run-example org.apache.spark.examples.ml.MinHashLSHExample + */ object MinHashLSHExample { def main(args: Array[String]): Unit = { // Creates a SparkSession @@ -73,7 +78,8 @@ object MinHashLSHExample { // 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 because of lack of elements in the hash buckets. + // It may return less than 2 rows when not enough approximate near-neighbor candidates are + // found. println("Approximately searching dfA for 2 nearest neighbors of the key:") model.approxNearestNeighbors(dfA, key, 2).show() // $example off$ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala index ebb969f99cbf2..cbac16345a292 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala @@ -111,8 +111,8 @@ class BucketedRandomProjectionLSHModel private[ml]( * Euclidean distance metrics. * * The input is dense or sparse vectors, each of which represents a point in the Euclidean - * distance space. The output will be vectors of configurable dimension. Hash values in the same - * dimension are calculated by the same hash function. + * distance space. The output will be vectors of configurable dimension. Hash values in the + * same dimension are calculated by the same hash function. * * References: * diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index af452b5f69fd6..a0edef50eebd5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -230,9 +230,10 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] * @param datasetA One of the datasets to join. * @param datasetB Another dataset to join. * @param threshold The threshold for the distance of row pairs. - * @param distCol Output column for storing the distance between each result row and the key. + * @param distCol Output column for storing the distance between each pair of rows. * @return A joined dataset containing pairs of rows. The original rows are in columns - * "datasetA" and "datasetB", and a distCol is added to show the distance of each pair. + * "datasetA" and "datasetB", and a distCol is added to show the distance between each + * pair. */ def approxSimilarityJoin( datasetA: Dataset[_], diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9833d35f7d2b1..19cc44ee1a406 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -135,14 +135,12 @@ class LSHParams(Params): def __init__(self): super(LSHParams, self).__init__() - @since("2.2.0") def setNumHashTables(self, value): """ Sets the value of :py:attr:`numHashTables`. """ return self._set(numHashTables=value) - @since("2.2.0") def getNumHashTables(self): """ Gets the value of numHashTables or its default value. @@ -155,7 +153,6 @@ class LSHModel(JavaModel): Mixin for Locality Sensitive Hashing (LSH) models. """ - @since("2.2.0") def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, distCol="distCol"): """ Given a large dataset and an item, approximately find at most k items which have the @@ -170,13 +167,12 @@ def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, distCol="dis :param numNearestNeighbors: The maximum number of nearest neighbors. :param distCol: Output column for storing the distance between each result row and the key. Use "distCol" as default value if it's not specified. - :return: A dataset containing at most k items closest to the key. A distCol is added - to show the distance between each row and the key. + :return: A dataset containing at most k items closest to the key. A column "distCol" is + added to show the distance between each row and the key. """ return self._call_java("approxNearestNeighbors", dataset, key, numNearestNeighbors, distCol) - @since("2.2.0") def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol"): """ Join two datasets to approximately find all pairs of rows whose distance are smaller than @@ -187,11 +183,11 @@ def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol") :param datasetA: One of the datasets to join. :param datasetB: Another dataset to join. :param threshold: The threshold for the distance of row pairs. - :param distCol: Output column for storing the distance between each result row and the key. - Use "distCol" as default value if it's not specified. + :param distCol: Output column for storing the distance between each pair of rows. Use + "distCol" as default value if it's not specified. :return: A joined dataset containing pairs of rows. The original rows are in columns - "datasetA" and "datasetB", and a distCol is added to show the distance of - each pair. + "datasetA" and "datasetB", and a column "distCol" is added to show the distance + between each pair. """ return self._call_java("approxSimilarityJoin", datasetA, datasetB, threshold, distCol) @@ -230,10 +226,10 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp >>> df2 = spark.createDataFrame(data2, ["id", "features"]) >>> model.approxNearestNeighbors(df2, Vectors.dense([1.0, 2.0]), 1).collect() [Row(id=4, features=DenseVector([2.0, 2.0]), hashes=[DenseVector([1.0])], distCol=1.0)] - >>> model.approxSimilarityJoin(df, df2, 3.0).select( + >>> model.approxSimilarityJoin(df, df2, 3.0, distCol="EuclideanDistance").select( ... col("datasetA.id").alias("idA"), ... col("datasetB.id").alias("idB"), - ... col("distCol").alias("EuclideanDistance")).show() + ... col("EuclideanDistance")).show() +---+---+-----------------+ |idA|idB|EuclideanDistance| +---+---+-----------------+ @@ -980,10 +976,10 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, >>> key = Vectors.sparse(6, [1, 2], [1.0, 1.0]) >>> model.approxNearestNeighbors(df2, key, 1).collect() [Row(id=5, features=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), hashes=[DenseVector([-163892... - >>> model.approxSimilarityJoin(df, df2, 0.6).select( + >>> model.approxSimilarityJoin(df, df2, 0.6, distCol="JaccardDistance").select( ... col("datasetA.id").alias("idA"), ... col("datasetB.id").alias("idB"), - ... col("distCol").alias("JaccardDistance")).show() + ... col("JaccardDistance")).show() +---+---+---------------+ |idA|idB|JaccardDistance| +---+---+---------------+ From 5d55752024d6db2493f7d3291ef774d767040121 Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Tue, 14 Feb 2017 09:12:57 -0800 Subject: [PATCH 19/20] Code Review Comments: Some minor fixes --- .../examples/ml/BucketedRandomProjectionLSHExample.scala | 2 +- .../org/apache/spark/examples/ml/MinHashLSHExample.scala | 6 +++--- mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala | 4 ++-- python/pyspark/ml/feature.py | 4 ++++ 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala index cf80659c24246..654535c264a35 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.BucketedRandomProjectionLSH import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.functions.col // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala index 92a08e778b6b0..6c1e22268ad2c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.MinHashLSH import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.functions.col // $example off$ import org.apache.spark.sql.SparkSession @@ -69,10 +69,10 @@ object MinHashLSHExample { // We could avoid computing hashes by passing in the already-transformed dataset, e.g. // `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` println("Approximately joining dfA and dfB on Jaccard distance smaller than 0.6:") - model.approxSimilarityJoin(dfA, dfB, 0.6) + model.approxSimilarityJoin(dfA, dfB, 0.6, "JaccardDistance") .select(col("datasetA.id").alias("idA"), col("datasetB.id").alias("idB"), - col("distCol").alias("JaccardDistance")).show() + col("JaccardDistance")).show() // Compute the locality sensitive hashes for the input rows, then perform approximate nearest // neighbor search. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index a0edef50eebd5..1c9f47a0b201d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -232,8 +232,8 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] * @param threshold The threshold for the distance of row pairs. * @param distCol Output column for storing the distance between each pair of rows. * @return A joined dataset containing pairs of rows. The original rows are in columns - * "datasetA" and "datasetB", and a distCol is added to show the distance between each - * pair. + * "datasetA" and "datasetB", and a column "distCol" is added to show the distance + * between each pair. */ def approxSimilarityJoin( datasetA: Dataset[_], diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 19cc44ee1a406..55d97ef1986f9 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -244,6 +244,8 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp >>> modelPath = temp_path + "/brp-model" >>> model.save(modelPath) >>> model2 = BucketedRandomProjectionLSHModel.load(modelPath) + >>> model.transform(df).head().hashes == model2.transform(df).head().hashes + True .. versionadded:: 2.2.0 """ @@ -995,6 +997,8 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed, >>> modelPath = temp_path + "/mh-model" >>> model.save(modelPath) >>> model2 = MinHashLSHModel.load(modelPath) + >>> model.transform(df).head().hashes == model2.transform(df).head().hashes + True .. versionadded:: 2.2.0 """ From d849c3a3d48be5d7ee90edfd8569f7b3fed3848f Mon Sep 17 00:00:00 2001 From: Yun Ni Date: Tue, 14 Feb 2017 23:21:18 -0800 Subject: [PATCH 20/20] Code Review Comment --- .../org/apache/spark/examples/ml/JavaMinHashLSHExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java index e36e5ba1e3445..0aace46939257 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -17,7 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.SparkSession; // $example on$ @@ -26,6 +25,7 @@ import org.apache.spark.ml.feature.MinHashLSH; import org.apache.spark.ml.feature.MinHashLSHModel; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset;