zhipeng93 commented on code in PR #192: URL: https://github.com/apache/flink-ml/pull/192#discussion_r1093002998
########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.ml.common.param.HasOutputCol; +import org.apache.flink.ml.param.DoubleParam; +import org.apache.flink.ml.param.IntParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidators; +import org.apache.flink.ml.param.StringParam; +import org.apache.flink.ml.param.WithParams; + +/** + * Params for {@link Swing}. + * + * @param <T> The class type of this instance. + */ +public interface SwingParams<T> extends WithParams<T>, HasOutputCol<T> { + Param<String> USER_COL = + new StringParam("userCol", "Name of user column.", "user", ParamValidators.notNull()); + + Param<String> ITEM_COL = + new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull()); + + Param<Integer> MAX_ITEM_USERS = + new IntParam( + "maxItemUsers", + "The max number of users used by Swing algorithm. If an item has users more than this value, Swing " + + "will sample pat of users.", + 1000, + ParamValidators.gt(0)); + + Param<Integer> K = + new IntParam( + "k", + "The max number of related items for each item.", + 100, + ParamValidators.gt(0)); + + Param<Integer> MIN_USER_ITEMS = + new IntParam( + "minUserItems", + "The min number of user-item interactions of a user.", + 10, + ParamValidators.gt(0)); + + Param<Integer> MAX_USER_ITEMS = Review Comment: Let's update the parameter name as `MAX_USER_BEHAVIOR` and also update the description. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.ml.common.param.HasOutputCol; +import org.apache.flink.ml.param.DoubleParam; +import org.apache.flink.ml.param.IntParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidators; +import org.apache.flink.ml.param.StringParam; +import org.apache.flink.ml.param.WithParams; + +/** + * Params for {@link Swing}. + * + * @param <T> The class type of this instance. + */ +public interface SwingParams<T> extends WithParams<T>, HasOutputCol<T> { + Param<String> USER_COL = + new StringParam("userCol", "Name of user column.", "user", ParamValidators.notNull()); + + Param<String> ITEM_COL = + new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull()); + + Param<Integer> MAX_ITEM_USERS = + new IntParam( + "maxItemUsers", + "The max number of users used by Swing algorithm. If an item has users more than this value, Swing " + + "will sample pat of users.", + 1000, + ParamValidators.gt(0)); + + Param<Integer> K = + new IntParam( + "k", + "The max number of related items for each item.", Review Comment: nit: Let's update the description as: `The max number of similar items to output for each item.` ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.ml.common.param.HasOutputCol; +import org.apache.flink.ml.param.DoubleParam; +import org.apache.flink.ml.param.IntParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidators; +import org.apache.flink.ml.param.StringParam; +import org.apache.flink.ml.param.WithParams; + +/** + * Params for {@link Swing}. + * + * @param <T> The class type of this instance. + */ +public interface SwingParams<T> extends WithParams<T>, HasOutputCol<T> { + Param<String> USER_COL = + new StringParam("userCol", "Name of user column.", "user", ParamValidators.notNull()); + + Param<String> ITEM_COL = + new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull()); + + Param<Integer> MAX_ITEM_USERS = Review Comment: The naming could be more intuitive here. How about renaming the parameter as `MAX_USER_NUM_PER_ITEM` and updating the description as follows? The max number of users that has purchased for each item. If the number of users that have purchased this item is larger than this value, then only `maxUserNumPerItem` users will be sampled and used in the computation logic. ########## flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java: ########## @@ -0,0 +1,226 @@ +/* + * 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.recommendation; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.recommendation.swing.Swing; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +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 org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +/** Tests {@link Swing}. */ +public class SwingTest { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + private Table trainData; + private static final List<Row> trainRows = + new ArrayList<>( + Arrays.asList( + Row.of(0L, 10L), + Row.of(0L, 11L), + Row.of(0L, 12L), + Row.of(1L, 13L), + Row.of(1L, 12L), + Row.of(2L, 10L), + Row.of(2L, 11L), + Row.of(2L, 12L), + Row.of(3L, 13L), + Row.of(3L, 12L))); + + private static final List<Row> expectedScoreRows = + new ArrayList<>( + Arrays.asList( + Row.of(10L, "11,0.058845768947156235;12,0.058845768947156235"), + Row.of(11L, "10,0.058845768947156235;12,0.058845768947156235"), + Row.of( + 12L, + "13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235"), + Row.of(13L, "12,0.09134833828228624"))); + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.getConfig().enableObjectReuse(); + env.setParallelism(2); Review Comment: Let's set the default parallelism as four following existing conventions. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.ml.common.param.HasOutputCol; +import org.apache.flink.ml.param.DoubleParam; +import org.apache.flink.ml.param.IntParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidators; +import org.apache.flink.ml.param.StringParam; +import org.apache.flink.ml.param.WithParams; + +/** + * Params for {@link Swing}. + * + * @param <T> The class type of this instance. + */ +public interface SwingParams<T> extends WithParams<T>, HasOutputCol<T> { + Param<String> USER_COL = + new StringParam("userCol", "Name of user column.", "user", ParamValidators.notNull()); + + Param<String> ITEM_COL = + new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull()); + + Param<Integer> MAX_ITEM_USERS = + new IntParam( + "maxItemUsers", + "The max number of users used by Swing algorithm. If an item has users more than this value, Swing " + + "will sample pat of users.", + 1000, + ParamValidators.gt(0)); + + Param<Integer> K = + new IntParam( + "k", + "The max number of related items for each item.", + 100, + ParamValidators.gt(0)); + + Param<Integer> MIN_USER_ITEMS = + new IntParam( + "minUserItems", + "The min number of user-item interactions of a user.", + 10, + ParamValidators.gt(0)); + + Param<Integer> MAX_USER_ITEMS = + new IntParam( + "maxUserItems", + "The max number of user-item interactions of a user.", + 1000, + ParamValidators.gt(0)); + + Param<Integer> ALPHA1 = + new IntParam( + "alpha1", + "This parameter is used to calculate weight of each user.", Review Comment: Could you update the description as follows and explains how it would affect the swing result? e.g., parameter of Swing. The higher alpha1 is, the more/less ... Same for alpha2 and beta. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java: ########## @@ -0,0 +1,455 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.iteration.operator.OperatorStateUtils; +import org.apache.flink.ml.api.Transformer; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.table.catalog.ResolvedSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; + +/** + * An Transformer which implements the Swing algorithm. + * + * <p>Swing is an item recall model. The topology of user-item graph usually can be described as + * user-item-user or item-user-item, which are like 'swing'. For example, if both user <em>u</em> + * and user <em>v</em> have purchased the same commodity <em>i</em> , they will form a relationship + * diagram similar to a swing. If <em>u</em> and <em>v</em> have purchased commodity <em>j</em> in + * addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are similar. + */ +public class Swing implements Transformer<Swing>, SwingParams<Swing> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + + public Swing() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public Table[] transform(Table... inputs) { + + final String userCol = getUserCol(); + final String itemCol = getItemCol(); + Preconditions.checkArgument(inputs.length == 1); + final ResolvedSchema schema = inputs[0].getResolvedSchema(); + + if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol)) + && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) { + throw new IllegalArgumentException("The types of user and item columns must be Long."); + } + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + + SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers = + tEnv.toDataStream(inputs[0]) + .map( + row -> { + if (row.getFieldAs(userCol) == null + || row.getFieldAs(itemCol) == null) { + throw new RuntimeException( + "Data of user and item column must not be null"); + } + return Tuple2.of( + ((Number) row.getFieldAs(userCol)).longValue(), + ((Number) row.getFieldAs(itemCol)).longValue()); + }) + .returns(Types.TUPLE(Types.LONG, Types.LONG)); + + SingleOutputStreamOperator<Tuple3<Long, Long, List<Long>>> userAllItemsStream = + itemUsers + .keyBy(tuple -> tuple.f0) + .transform( + "fillUserItemsTable", + Types.TUPLE(Types.LONG, Types.LONG, Types.LIST(Types.LONG)), + new BuildSwingData(getMinUserItems(), getMaxUserItems())); + + RowTypeInfo outputTypeInfo = + new RowTypeInfo( + new TypeInformation[] { + BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + }, + new String[] {getItemCol(), getOutputCol()}); + + DataStream<Row> output = + userAllItemsStream + .keyBy(tuple -> tuple.f1) + .transform( + "calculateSwingSimilarity", + new TupleTypeInfo<>( + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO), + new CalculateSimilarity( + getK(), + getMaxItemUsers(), + getAlpha1(), + getAlpha2(), + getBeta())) + .map( + new MapFunction<Tuple2<Long, String>, Row>() { Review Comment: The `map` seems unnecessary here. Can you explain why it is needed here? ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.ml.common.param.HasOutputCol; +import org.apache.flink.ml.param.DoubleParam; +import org.apache.flink.ml.param.IntParam; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.param.ParamValidators; +import org.apache.flink.ml.param.StringParam; +import org.apache.flink.ml.param.WithParams; + +/** + * Params for {@link Swing}. + * + * @param <T> The class type of this instance. + */ +public interface SwingParams<T> extends WithParams<T>, HasOutputCol<T> { + Param<String> USER_COL = + new StringParam("userCol", "Name of user column.", "user", ParamValidators.notNull()); + + Param<String> ITEM_COL = + new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull()); + + Param<Integer> MAX_ITEM_USERS = + new IntParam( + "maxItemUsers", + "The max number of users used by Swing algorithm. If an item has users more than this value, Swing " + + "will sample pat of users.", + 1000, + ParamValidators.gt(0)); + + Param<Integer> K = + new IntParam( + "k", + "The max number of related items for each item.", + 100, + ParamValidators.gt(0)); + + Param<Integer> MIN_USER_ITEMS = Review Comment: Let's update the parameter name as `MIN_USER_BEHAVIOR` and also update the description. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java: ########## @@ -0,0 +1,455 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.iteration.operator.OperatorStateUtils; +import org.apache.flink.ml.api.Transformer; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.table.catalog.ResolvedSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; + +/** + * An Transformer which implements the Swing algorithm. + * + * <p>Swing is an item recall model. The topology of user-item graph usually can be described as + * user-item-user or item-user-item, which are like 'swing'. For example, if both user <em>u</em> + * and user <em>v</em> have purchased the same commodity <em>i</em> , they will form a relationship + * diagram similar to a swing. If <em>u</em> and <em>v</em> have purchased commodity <em>j</em> in + * addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are similar. + */ +public class Swing implements Transformer<Swing>, SwingParams<Swing> { Review Comment: Let's implement `Swing` as an AlgoOperator. The difference between AlgoOperator and Transformer is that AlgoOperator can express the aggregation semantic: - Data transformation (Transformer): each output record is computed from a single input record - Data aggregation (AlgoOperator): each output record is computed using more than one input record. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java: ########## @@ -0,0 +1,455 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.iteration.operator.OperatorStateUtils; +import org.apache.flink.ml.api.Transformer; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.table.catalog.ResolvedSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; + +/** + * An Transformer which implements the Swing algorithm. + * + * <p>Swing is an item recall model. The topology of user-item graph usually can be described as + * user-item-user or item-user-item, which are like 'swing'. For example, if both user <em>u</em> + * and user <em>v</em> have purchased the same commodity <em>i</em> , they will form a relationship + * diagram similar to a swing. If <em>u</em> and <em>v</em> have purchased commodity <em>j</em> in + * addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are similar. + */ +public class Swing implements Transformer<Swing>, SwingParams<Swing> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + + public Swing() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public Table[] transform(Table... inputs) { + + final String userCol = getUserCol(); + final String itemCol = getItemCol(); + Preconditions.checkArgument(inputs.length == 1); + final ResolvedSchema schema = inputs[0].getResolvedSchema(); + + if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol)) + && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) { + throw new IllegalArgumentException("The types of user and item columns must be Long."); + } + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + + SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers = + tEnv.toDataStream(inputs[0]) + .map( + row -> { + if (row.getFieldAs(userCol) == null + || row.getFieldAs(itemCol) == null) { + throw new RuntimeException( + "Data of user and item column must not be null"); + } + return Tuple2.of( + ((Number) row.getFieldAs(userCol)).longValue(), + ((Number) row.getFieldAs(itemCol)).longValue()); + }) + .returns(Types.TUPLE(Types.LONG, Types.LONG)); + + SingleOutputStreamOperator<Tuple3<Long, Long, List<Long>>> userAllItemsStream = + itemUsers + .keyBy(tuple -> tuple.f0) + .transform( + "fillUserItemsTable", + Types.TUPLE(Types.LONG, Types.LONG, Types.LIST(Types.LONG)), + new BuildSwingData(getMinUserItems(), getMaxUserItems())); + + RowTypeInfo outputTypeInfo = + new RowTypeInfo( + new TypeInformation[] { + BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + }, + new String[] {getItemCol(), getOutputCol()}); + + DataStream<Row> output = + userAllItemsStream + .keyBy(tuple -> tuple.f1) + .transform( + "calculateSwingSimilarity", + new TupleTypeInfo<>( + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO), + new CalculateSimilarity( Review Comment: Would it be more intuitive to name it as `ComputingSimilarItems`? ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java: ########## @@ -0,0 +1,455 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.iteration.operator.OperatorStateUtils; +import org.apache.flink.ml.api.Transformer; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.table.catalog.ResolvedSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; + +/** + * An Transformer which implements the Swing algorithm. + * + * <p>Swing is an item recall model. The topology of user-item graph usually can be described as + * user-item-user or item-user-item, which are like 'swing'. For example, if both user <em>u</em> + * and user <em>v</em> have purchased the same commodity <em>i</em> , they will form a relationship + * diagram similar to a swing. If <em>u</em> and <em>v</em> have purchased commodity <em>j</em> in + * addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are similar. + */ +public class Swing implements Transformer<Swing>, SwingParams<Swing> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + + public Swing() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public Table[] transform(Table... inputs) { + + final String userCol = getUserCol(); + final String itemCol = getItemCol(); + Preconditions.checkArgument(inputs.length == 1); + final ResolvedSchema schema = inputs[0].getResolvedSchema(); + + if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol)) + && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) { + throw new IllegalArgumentException("The types of user and item columns must be Long."); + } + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + + SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers = + tEnv.toDataStream(inputs[0]) + .map( + row -> { + if (row.getFieldAs(userCol) == null + || row.getFieldAs(itemCol) == null) { + throw new RuntimeException( + "Data of user and item column must not be null"); + } + return Tuple2.of( + ((Number) row.getFieldAs(userCol)).longValue(), + ((Number) row.getFieldAs(itemCol)).longValue()); + }) + .returns(Types.TUPLE(Types.LONG, Types.LONG)); + + SingleOutputStreamOperator<Tuple3<Long, Long, List<Long>>> userAllItemsStream = + itemUsers + .keyBy(tuple -> tuple.f0) + .transform( + "fillUserItemsTable", + Types.TUPLE(Types.LONG, Types.LONG, Types.LIST(Types.LONG)), + new BuildSwingData(getMinUserItems(), getMaxUserItems())); Review Comment: Would `CollectingUserBehavior` be more intuitive? ########## flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java: ########## @@ -0,0 +1,226 @@ +/* + * 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.recommendation; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.recommendation.swing.Swing; +import org.apache.flink.ml.util.TestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +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 org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +/** Tests {@link Swing}. */ +public class SwingTest { + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + private Table trainData; + private static final List<Row> trainRows = + new ArrayList<>( + Arrays.asList( + Row.of(0L, 10L), + Row.of(0L, 11L), + Row.of(0L, 12L), + Row.of(1L, 13L), + Row.of(1L, 12L), + Row.of(2L, 10L), + Row.of(2L, 11L), + Row.of(2L, 12L), + Row.of(3L, 13L), + Row.of(3L, 12L))); + + private static final List<Row> expectedScoreRows = + new ArrayList<>( + Arrays.asList( + Row.of(10L, "11,0.058845768947156235;12,0.058845768947156235"), + Row.of(11L, "10,0.058845768947156235;12,0.058845768947156235"), + Row.of( + 12L, + "13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235"), + Row.of(13L, "12,0.09134833828228624"))); + + @Before + public void before() { + Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.getConfig().enableObjectReuse(); + env.setParallelism(2); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.noRestart()); + tEnv = StreamTableEnvironment.create(env); + Schema schema = + Schema.newBuilder() + .column("f0", DataTypes.BIGINT()) Review Comment: It could be simplified by replacing `f0` and `f1` with `user_id` and `item_id` here. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java: ########## @@ -0,0 +1,455 @@ +/* + * 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.recommendation.swing; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.iteration.operator.OperatorStateUtils; +import org.apache.flink.ml.api.Transformer; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +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.table.catalog.ResolvedSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; + +/** + * An Transformer which implements the Swing algorithm. + * + * <p>Swing is an item recall model. The topology of user-item graph usually can be described as + * user-item-user or item-user-item, which are like 'swing'. For example, if both user <em>u</em> + * and user <em>v</em> have purchased the same commodity <em>i</em> , they will form a relationship + * diagram similar to a swing. If <em>u</em> and <em>v</em> have purchased commodity <em>j</em> in + * addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are similar. + */ +public class Swing implements Transformer<Swing>, SwingParams<Swing> { Review Comment: Also, could you add the mathmatical formula of swing here? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
