@@ -224,7 +224,8 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp
224224 ... (Vectors.dense([1.0, -1.0 ]),),
225225 ... (Vectors.dense([1.0, 1.0]),)]
226226 >>> df = spark.createDataFrame(data, ["keys"])
227- >>> rp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values", seed=12345, bucketLength=1.0)
227+ >>> rp = BucketedRandomProjectionLSH(inputCol="keys", outputCol="values",
228+ ... seed=12345, bucketLength=1.0)
228229 >>> model = rp.fit(df)
229230 >>> model.randUnitVectors
230231 [DenseVector([-0.3041, 0.9527])]
@@ -258,22 +259,26 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp
258259 typeConverter = TypeConverters .toFloat )
259260
260261 @keyword_only
261- def __init__ (self , inputCol = None , outputCol = None , seed = None , numHashTables = 1 , bucketLength = None ):
262+ def __init__ (self , inputCol = None , outputCol = None , seed = None , numHashTables = 1 ,
263+ bucketLength = None ):
262264 """
263- __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None)
265+ __init__(self, inputCol=None, outputCol=None, seed=None, numHashTables=1,
266+ bucketLength=None)
264267 """
265268 super (BucketedRandomProjectionLSH , self ).__init__ ()
266- self ._java_obj = self ._new_java_obj ("org.apache.spark.ml.feature.BucketedRandomProjectionLSH" ,
267- self .uid )
269+ self ._java_obj = self ._new_java_obj (
270+ "org.apache.spark.ml.feature.BucketedRandomProjectionLSH" , self .uid )
268271 self ._setDefault (numHashTables = 1 )
269272 kwargs = self .__init__ ._input_kwargs
270273 self .setParams (** kwargs )
271274
272275 @keyword_only
273276 @since ("2.2.0" )
274- def setParams (self , inputCol = None , outputCol = None , seed = None , numHashTables = 1 , bucketLength = None ):
277+ def setParams (self , inputCol = None , outputCol = None , seed = None , numHashTables = 1 ,
278+ bucketLength = None ):
275279 """
276- setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1, bucketLength=None)
280+ setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1,
281+ bucketLength=None)
277282 Sets params for this BucketedRandomProjectionLSH.
278283 """
279284 kwargs = self .setParams ._input_kwargs
@@ -301,10 +306,10 @@ class BucketedRandomProjectionLSHModel(JavaModel, LSHModel, JavaMLReadable, Java
301306 """
302307 .. note:: Experimental
303308
304- Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are stored.
305- The vectors are normalized to be unit vectors and each vector is used in a hash function:
306- : math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random unit
307- vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`.
309+ Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are
310+ stored. The vectors are normalized to be unit vectors and each vector is used in a hash
311+ function: : math:`h_i(x) = floor(r_i * x / bucketLength)` where :math:`r_i` is the i-th random
312+ unit vector. The number of buckets will be `(max L2 norm of input vectors) / bucketLength`.
308313
309314 .. versionadded:: 2.2.0
310315 """
@@ -318,7 +323,6 @@ def randUnitVectors(self):
318323 return self ._call_java ("randUnitVectors" )
319324
320325
321-
322326@inherit_doc
323327class Bucketizer (JavaTransformer , HasInputCol , HasOutputCol , JavaMLReadable , JavaMLWritable ):
324328 """
@@ -983,7 +987,7 @@ class MinHashLSH(JavaEstimator, LSHParams, HasInputCol, HasOutputCol, HasSeed,
983987 ... (Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)]
984988 >>> df2 = spark.createDataFrame(data2, ["keys"])
985989 >>> model.approxNearestNeighbors(df2, Vectors.sparse(6, [1], [1.0]), 1).collect()
986- [Row(keys=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), values=[DenseVector([-1638925712.0])], distCol=0.666 ...)]
990+ [Row(keys=SparseVector(6, {1: 1.0, 2: 1.0, 4: 1.0}), values=[DenseVector([-1638925712.0])]...)]
987991 >>> model.approxSimilarityJoin(df, df2, 1.0).select("distCol").head()[0]
988992 0.5
989993 >>> mhPath = temp_path + "/mh"
@@ -1032,8 +1036,8 @@ class MinHashLSHModel(JavaModel, LSHModel, JavaMLReadable, JavaMLWritable):
10321036 randomly chosen integers less than prime:`h_i(x) = ((x \cdot a_i + b_i) \mod prime)`
10331037 This hash family is approximately min-wise independent according to the reference.
10341038
1035- .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear permutations."
1036- Electronic Journal of Combinatorics 7 (2000): R26.
1039+ .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear
1040+ permutations." Electronic Journal of Combinatorics 7 (2000): R26.
10371041
10381042 .. versionadded:: 2.2.0
10391043 """
0 commit comments