[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #191: [FLINK-30401] Add Estimator and Transformer for MinHashLSH
jiangxin369 commented on code in PR #191: URL: https://github.com/apache/flink-ml/pull/191#discussion_r1060307196 ## flink-ml-python/pyflink/ml/lib/feature/lsh.py: ## @@ -0,0 +1,191 @@ + +# 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. + + +import typing +from abc import ABC +from pyflink.java_gateway import get_gateway +from pyflink.table import Table +from pyflink.util.java_utils import to_jarray + +from pyflink.ml.core.linalg import Vector, DenseVector, SparseVector +from pyflink.ml.core.param import Param, IntParam, ParamValidators +from pyflink.ml.core.wrapper import JavaWithParams +from pyflink.ml.lib.feature.common import JavaFeatureEstimator, JavaFeatureModel +from pyflink.ml.lib.param import HasInputCol, HasOutputCol, HasSeed + + +class _LSHModelParams(JavaWithParams, + HasInputCol, + HasOutputCol): +""" +Params for :class:`LSHModel` +""" + +def __init__(self, java_params): +super(_LSHModelParams, self).__init__(java_params) + + +class _LSHParams(_LSHModelParams): +""" +Params for :class:`LSH` +""" + +NUM_HASH_TABLES: Param[int] = IntParam( +"num_hash_tables", "Number of hash tables.", 1, ParamValidators.gt_eq(1) +) + +NUM_HASH_FUNCTIONS_PER_TABLE: Param[int] = IntParam( +"num_hash_functions_per_table", +"Number of hash functions per table.", +1, +ParamValidators.gt_eq(1.)) + +def __init__(self, java_params): +super(_LSHParams, self).__init__(java_params) + +def set_num_hash_tables(self, value: int): +return typing.cast(_LSHParams, self.set(self.NUM_HASH_TABLES, value)) + +def get_num_hash_tables(self): +return self.get(self.NUM_HASH_TABLES) + +@property +def num_hash_tables(self): +return self.get_num_hash_tables() + +def set_num_hash_functions_per_table(self, value: int): +return typing.cast(_LSHParams, self.set(self.NUM_HASH_FUNCTIONS_PER_TABLE, value)) + +def get_num_hash_functions_per_table(self): +return self.get(self.NUM_HASH_FUNCTIONS_PER_TABLE) + +@property +def num_hash_functions_per_table(self): +return self.get_num_hash_functions_per_table() + + +class _LSH(JavaFeatureEstimator, ABC): +""" +Base class for estimators which implement LSH (Locality-sensitive hashing) algorithms. +""" + +def __init__(self): +super(_LSH, self).__init__() + +@classmethod +def _java_estimator_package_name(cls) -> str: +return "lsh" + + +class _LSHModel(JavaFeatureModel, ABC): +""" +Base class for LSH model. +""" + +def __init__(self, java_model): +super(_LSHModel, self).__init__(java_model) + +@classmethod +def _java_model_package_name(cls) -> str: +return "lsh" + +def approx_nearest_neighbors(self, dataset: Table, key: Vector, k: int, + dist_col: str = 'distCol'): +""" +Given a dataset and an item, approximately find at most k items which have the closest +distance to the item. If the `outputCol` is missing in the given dataset, this method +transforms the dataset with the model at first. + +:param dataset: The dataset in which to to search for nearest neighbors. +:param key: The item to search for. +:param k: The maximum number of nearest neighbors. +:param dist_col: The output column storing the distance between each neighbor and the +key. +:return: A dataset containing at most k items closest to the key with a column named +`distCol` appended. Review Comment: To keep the same coding style with PyFlink and flink-ml-python, it's recommended to align the beginning of each line. E.g., ``` :param dist_col: The output column storing the distance between each neighbor and the key. :return: A dataset containing at most k items closest to the key with a column named `distCol` appended. ``` So as the other com
[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #191: [FLINK-30401] Add Estimator and Transformer for MinHashLSH
jiangxin369 commented on code in PR #191: URL: https://github.com/apache/flink-ml/pull/191#discussion_r1060307196 ## flink-ml-python/pyflink/ml/lib/feature/lsh.py: ## @@ -0,0 +1,191 @@ + +# 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. + + +import typing +from abc import ABC +from pyflink.java_gateway import get_gateway +from pyflink.table import Table +from pyflink.util.java_utils import to_jarray + +from pyflink.ml.core.linalg import Vector, DenseVector, SparseVector +from pyflink.ml.core.param import Param, IntParam, ParamValidators +from pyflink.ml.core.wrapper import JavaWithParams +from pyflink.ml.lib.feature.common import JavaFeatureEstimator, JavaFeatureModel +from pyflink.ml.lib.param import HasInputCol, HasOutputCol, HasSeed + + +class _LSHModelParams(JavaWithParams, + HasInputCol, + HasOutputCol): +""" +Params for :class:`LSHModel` +""" + +def __init__(self, java_params): +super(_LSHModelParams, self).__init__(java_params) + + +class _LSHParams(_LSHModelParams): +""" +Params for :class:`LSH` +""" + +NUM_HASH_TABLES: Param[int] = IntParam( +"num_hash_tables", "Number of hash tables.", 1, ParamValidators.gt_eq(1) +) + +NUM_HASH_FUNCTIONS_PER_TABLE: Param[int] = IntParam( +"num_hash_functions_per_table", +"Number of hash functions per table.", +1, +ParamValidators.gt_eq(1.)) + +def __init__(self, java_params): +super(_LSHParams, self).__init__(java_params) + +def set_num_hash_tables(self, value: int): +return typing.cast(_LSHParams, self.set(self.NUM_HASH_TABLES, value)) + +def get_num_hash_tables(self): +return self.get(self.NUM_HASH_TABLES) + +@property +def num_hash_tables(self): +return self.get_num_hash_tables() + +def set_num_hash_functions_per_table(self, value: int): +return typing.cast(_LSHParams, self.set(self.NUM_HASH_FUNCTIONS_PER_TABLE, value)) + +def get_num_hash_functions_per_table(self): +return self.get(self.NUM_HASH_FUNCTIONS_PER_TABLE) + +@property +def num_hash_functions_per_table(self): +return self.get_num_hash_functions_per_table() + + +class _LSH(JavaFeatureEstimator, ABC): +""" +Base class for estimators which implement LSH (Locality-sensitive hashing) algorithms. +""" + +def __init__(self): +super(_LSH, self).__init__() + +@classmethod +def _java_estimator_package_name(cls) -> str: +return "lsh" + + +class _LSHModel(JavaFeatureModel, ABC): +""" +Base class for LSH model. +""" + +def __init__(self, java_model): +super(_LSHModel, self).__init__(java_model) + +@classmethod +def _java_model_package_name(cls) -> str: +return "lsh" + +def approx_nearest_neighbors(self, dataset: Table, key: Vector, k: int, + dist_col: str = 'distCol'): +""" +Given a dataset and an item, approximately find at most k items which have the closest +distance to the item. If the `outputCol` is missing in the given dataset, this method +transforms the dataset with the model at first. + +:param dataset: The dataset in which to to search for nearest neighbors. +:param key: The item to search for. +:param k: The maximum number of nearest neighbors. +:param dist_col: The output column storing the distance between each neighbor and the +key. +:return: A dataset containing at most k items closest to the key with a column named +`distCol` appended. Review Comment: To keep the same coding style with PyFlink and flink-ml-python, it's recommended to align the beginning of each line. E.g., ``` :param dist_col: The output column storing the distance between each neighbor and the key. :return: A dataset containing at most k items closest to the key with a column named `distCol` appended. ``` So as the other com
[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #191: [FLINK-30401] Add Estimator and Transformer for MinHashLSH
jiangxin369 commented on code in PR #191: URL: https://github.com/apache/flink-ml/pull/191#discussion_r1060261023 ## flink-ml-examples/src/main/java/org/apache/flink/ml/examples/feature/MinHashLSHExample.java: ## @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.ml.examples.feature; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.ml.feature.lsh.MinHashLSH; +import org.apache.flink.ml.feature.lsh.MinHashLSHModel; +import org.apache.flink.ml.linalg.DenseVector; +import org.apache.flink.ml.linalg.SparseVector; +import org.apache.flink.ml.linalg.Vector; +import org.apache.flink.ml.linalg.Vectors; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import org.apache.commons.collections.IteratorUtils; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.table.api.Expressions.$; + +/** + * Simple program that trains a MinHashLSH model and uses it for approximate nearest neighbors and + * similarity join. + */ +public class MinHashLSHExample { +public static void main(String[] args) throws Exception { + +// Creates a new StreamExecutionEnvironment +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + +// Creates a StreamTableEnvironment +StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + +// Generates two datasets +Table data = +tEnv.fromDataStream( +env.fromCollection( +Arrays.asList( +Row.of( +0, +Vectors.sparse( +6, +new int[] {0, 1, 2}, +new double[] {1., 1., 1.})), +Row.of( +1, +Vectors.sparse( +6, +new int[] {2, 3, 4}, +new double[] {1., 1., 1.})), +Row.of( +2, +Vectors.sparse( +6, +new int[] {0, 2, 4}, +new double[] {1., 1., 1.}))), +Types.ROW_NAMED( +new String[] {"id", "vec"}, +Types.INT, + TypeInformation.of(SparseVector.class; + +Table dataB = Review Comment: As the examples are expected to be the best practice for users, the variable naming should be more strict. Would it be better if rename the `data` to `inputTable`, `dataB` to `similarityJoinTable`? `dataA` and `dataB` are also acceptable. So as in the python example. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #191: [FLINK-30401] Add Estimator and Transformer for MinHashLSH
jiangxin369 commented on code in PR #191: URL: https://github.com/apache/flink-ml/pull/191#discussion_r1060256575 ## docs/content/docs/operators/feature/minhashlsh.md: ## @@ -0,0 +1,280 @@ +--- +title: "MinHash LSH" +weight: 1 +type: docs +aliases: +- /operators/feature/minhashlsh.html +--- + + + +## MinHash LSH + +MinHash LSH is a Locality Sensitive Hashing (LSH) scheme for Jaccard distance metric. +The input features are sets of natural numbers represented as non-zero indices of vectors, +either dense vectors or sparse vectors. Typically, sparse vectors are more efficient. + +In addition to transforming input feature vectors to multiple hash values, the MinHash LSH +model also supports approximate nearest neighbors search within a dataset regarding a key +vector and approximate similarity join between two datasets. + +### Input Columns + +| Param name | Type | Default | Description| +|:---|:---|:--|:---| +| inputCol | Vector | `"input"` | Features to be mapped. | + +### Output Columns + +| Param name | Type | Default| Description | +|:---|:--|:---|:-| +| outputCol | DenseVector[] | `"output"` | Hash values. | + +### Parameters Review Comment: Since `MinHashLSH` has some parameters that cannot be set to `MinHashLSHModel`, let's separate the parameters into two tables just like in `vectorindexer.md`. ## flink-ml-python/pyflink/ml/lib/feature/lsh.py: ## @@ -0,0 +1,191 @@ + +# 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. + + +import typing +from abc import ABC +from pyflink.java_gateway import get_gateway +from pyflink.table import Table +from pyflink.util.java_utils import to_jarray + +from pyflink.ml.core.linalg import Vector, DenseVector, SparseVector +from pyflink.ml.core.param import Param, IntParam, ParamValidators +from pyflink.ml.core.wrapper import JavaWithParams +from pyflink.ml.lib.feature.common import JavaFeatureEstimator, JavaFeatureModel +from pyflink.ml.lib.param import HasInputCol, HasOutputCol, HasSeed + + +class _LSHModelParams(JavaWithParams, + HasInputCol, + HasOutputCol): +""" +Params for :class:`LSHModel` +""" + +def __init__(self, java_params): +super(_LSHModelParams, self).__init__(java_params) + + +class _LSHParams(_LSHModelParams): +""" +Params for :class:`LSH` +""" + +NUM_HASH_TABLES: Param[int] = IntParam( +"num_hash_tables", "Number of hash tables.", 1, ParamValidators.gt_eq(1) +) + +NUM_HASH_FUNCTIONS_PER_TABLE: Param[int] = IntParam( +"num_hash_functions_per_table", +"Number of hash functions per table.", +1, +ParamValidators.gt_eq(1.)) Review Comment: nit: let's unify the parameter of `gt_eq` in `NUM_HASH_TABLES` and `NUM_HASH_FUNCTIONS_PER_TABLE` to either 1. or 1. ## flink-ml-examples/src/main/java/org/apache/flink/ml/examples/feature/MinHashLSHExample.java: ## @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.ml.examples.feature; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +impo
[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #191: [FLINK-30401] Add Estimator and Transformer for MinHashLSH
jiangxin369 commented on code in PR #191: URL: https://github.com/apache/flink-ml/pull/191#discussion_r1057228565 ## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/lsh/MinHashLSH.java: ## @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.ml.feature.lsh; + +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import java.io.IOException; + +/** + * An Estimator that implements the MinHash LSH algorithm, which supports LSH for Jaccard distance. + * + * The input could be dense or sparse vectors. Each input vector must hava at least one non-zero Review Comment: ```suggestion * The input could be dense or sparse vectors. Each input vector must have at least one non-zero ``` ## flink-ml-lib/src/main/java/org/apache/flink/ml/feature/lsh/LSHModel.java: ## @@ -0,0 +1,457 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.ml.feature.lsh; + +import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.ml.api.Model; +import org.apache.flink.ml.common.broadcast.BroadcastUtils; +import org.apache.flink.ml.common.datastream.DataStreamUtils; +import org.apache.flink.ml.common.datastream.EndOfStreamWindows; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.linalg.DenseVector; +import org.apache.flink.ml.linalg.Vector; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.ArrayUtils; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; + +/** + * Base class for LSH model. + * + * In addition to transforming input feature vectors to multiple hash values, it also supports + * approximate nearest neighbors search within a dataset regarding a key vector and approximate + * similarity join between two datasets. + * + * @param class type of the LSHModel implementation itself. + */ +abstract class LSHModel> implements Model, LSHModelParams { +private static final String MODEL_DATA_BC_KEY = "modelData"; + +private final Map, Object> paramMap = new HashMap<>(); + +/** Stores the corresponding model data class of T. */ +private final Class modelDataClass; + +protected Table modelDataTable; + +public LSHModel(Class modelDataClass) { +this.modelD