vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101074992


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,430 @@
+/*
+ * 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.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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/**
+ * An AlgoOperator 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. The formula of
+ * Swing is
+ *
+ * <p>$$ w_{(i,j)}=\sum_{u\in U_i\cap U_j}\sum_{v\in U_i\cap
+ * 
U_j}{\frac{1}{{(I_u+\alpha_1)}^\beta}}*{\frac{1}{{(I_v+\alpha_1)}^\beta}}*{\frac{1}{\alpha\_2+|I_u\cap
+ * I_v|}} $$
+ *
+ * <p>This implementation is based on the algorithm proposed in the paper: 
"Large Scale Product
+ * Graph Construction for Recommendation in E-commerce" by Xiaoyong Yang, 
Yadong Zhu and Yi Zhang.
+ * (https://arxiv.org/pdf/2010.05525.pdf)
+ */
+public class Swing implements AlgoOperator<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(
+                                            (Long) row.getFieldAs(userCol),
+                                            (Long) row.getFieldAs(itemCol));
+                                })
+                        .returns(Types.TUPLE(Types.LONG, Types.LONG));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Map<Long, String>>> 
userAllItemsStream =
+                itemUsers
+                        .keyBy(tuple -> tuple.f0)
+                        .transform(
+                                "fillUserItemsTable",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), 
getMaxUserBehavior()));
+
+        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(
+                                "computingSimilarItems",
+                                outputTypeInfo,
+                                new ComputingSimilarItems(
+                                        getK(),
+                                        getMaxUserNumPerItem(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        return new Table[] {tEnv.fromDataStream(output)};
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static Swing load(StreamTableEnvironment tEnv, String path) throws 
IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    /**
+     * Appends one column, that records all items the user has purchased, to 
the input table.
+     *
+     * <p>During the process, this operator collect users and all items a user 
has purchased into a
+     * map of list. When the input is finished, this operator appends the 
certain
+     * user-purchased-items list to each row.
+     */
+    private static class CollectingUserBehavior
+            extends AbstractStreamOperator<Tuple3<Long, Long, Map<Long, 
String>>>
+            implements OneInputStreamOperator<
+                            Tuple2<Long, Long>, Tuple3<Long, Long, Map<Long, 
String>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();

Review Comment:
   The reason to use `Map` instead of `Set` is to update`userAllItemsMapState` 
by value of `userItemsMap`. And type of `userAllItemsMapState` cannot be 
`ListState<Map<Long, Set<Long>>>` because Flink doesn't have TypeInformation of 
`Set` type.



-- 
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

Reply via email to