You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/12/19 08:49:09 UTC

[GitHub] [flink-ml] vacaly opened a new pull request, #192: [FLINK-30451] Add Estimator and Transformer for Swing

vacaly opened a new pull request, #192:
URL: https://github.com/apache/flink-ml/pull/192

   ## What is the purpose of the change
   
   * This pull request adds the implementation of Swing recommendation algorith.*
   
   ## Brief change log
   
     - *Adds Transformer and Estimator implementation of Swing recommendation algorithm in Java*
     - *Adds examples of Swing*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
   
   ## Documentation
   
   Swing[1] recommendation algorithm can measure the similarity between items by calculating their common users(or purchaser) and the difference of users. The Swing score is 
   $$score_(i,j)=\sum_{u \in U_i}\sum_{v \in U_j} \frac{1}{I_u\cap I_v}$$
   $U_i$ represents user set who clicked item $i$, $I_u$ represents item set that user $u$ clicked.
   
   Specifically, for the original input data of user-item click matrix, each row is a user and a clicked item, eg: "Danny, IPhone". 
    And the Swing estimator works as follow steps:
   1. At the first reducer stage, Swing estimator collects the item clicked list of the specific user, eg: "Danny, {IPhone, IPad, AirPods}". 
   2. At the next mapper stage, for each item clicked by a certain user. the estimator appends the user's item list to the row, eg: "Danny, IPhone, {IPhone, IPad, AirPods}".
   3. Then at the reducer stage, the estimator collects the user set $U_i$ who clicked item $i$ ,and the item set clicked by each user in $U_i$. Finally Swing score is calculated and save to SwingModel.
   
   
   [1]Large Scale Product Graph Construction for Recommendation in E-commerce,https://arxiv.org/pdf/2010.05525.pdf​​​​​​​


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101268759


##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static final Character commaDelimiter = ',';
+        private static final Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Map<Long, String> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers.keySet());
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers.keySet()) {

Review Comment:
   Thanks for your advice, `Collections.shuffle` is indeed better.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1070782561


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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;

Review Comment:
   `env, tenv and trainData` are global variables and accessed by several test functions.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053974995


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModelData.java:
##########
@@ -0,0 +1,106 @@
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.clustering.kmeans.KMeansModelData;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class SwingModelData {
+	public String mainItem;
+	public ArrayList <String> items;

Review Comment:
   Thanks for your suggestion. Using `List` results in `ClassCastException` in `SwingModelData.getDataStream()`. No idea why it can't work.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053958500


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						userItemsMap.put(k, itemSet);
+					});
+				});
+
+			itemUsersMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"itemUsersMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+				.ifPresent(x -> {
+					itemUsersMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						itemUsersMap.put(k, itemSet);
+					});
+				});
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				tmpUerItemsMap.put(k, new ArrayList <>(v));
+			});
+			userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));

Review Comment:
   Sorry that I missed the types by the lengthy conversion codes. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101090934


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
   Changed to:
   ```
   String itemList =
                           itemAndScore.stream()
                                   .sequential()
                                   .limit(itemNums)
                                   .map(tuple2 -> "" + tuple2.f0 + commaDelimiter + tuple2.f1)
                                   .collect(Collectors.joining("" + semicolonDelimiter));
   ```



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1099882916


##########
flink-ml-python/pyflink/ml/lib/recommendation/tests/test_swing.py:
##########
@@ -0,0 +1,165 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.table import Table
+from typing import List
+from py4j.protocol import Py4JJavaError:
+
+from pyflink.ml.lib.recommendation.swing import Swing
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+
+# Tests Swing. 
+class SwingTest(PyFlinkMLTestCase):
+    def setUp(self):
+        super(SwingTest, self).setUp()
+        self.train_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (0, 11),
+                (0, 12),
+                (1, 13),
+                (1, 12),
+                (2, 10),
+                (2, 11),
+                (2, 12),
+                (3, 13),
+                (3, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.wrong_type_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (1, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.INT(), Types.LONG()])
+            ))
+
+        self.none_value_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (None, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.expected_data = [
+            [10, '11,0.058845768947156235;12,0.058845768947156235'],
+            [11, '10,0.058845768947156235;12,0.058845768947156235'],
+            [12, '13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235'],
+            [13, '12,0.09134833828228624']]
+
+    def test_param(self):
+        swing = Swing()
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(100, swing.k)
+        self.assertEqual(10, swing.min_user_items)
+        self.assertEqual(1000, swing.max_user_items)
+        self.assertEqual(15, swing.alpha1)
+        self.assertEqual(0, swing.alpha2)
+        self.assertEqual(0.3, swing.beta, delta=1e-9)
+
+        swing.set_item_col("item_1") \
+            .set_user_col("user_1") \
+            .set_k(20) \
+            .set_min_user_behavior(20) \
+            .set_max_user_behavior(50) \
+            .set_alpha1(5) \
+            .set_alpha2(1) \
+            .set_beta(0.35)
+
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(20, swing.k)
+        self.assertEqual(20, swing.min_user_items)
+        self.assertEqual(50, swing.max_user_items)
+        self.assertEqual(5, swing.alpha1)
+        self.assertEqual(1, swing.alpha2)
+        self.assertEqual(0.35, swing.beta, delta=1e-9)
+
+    def test_output_schema(self):
+        swing = Swing() \
+            .set_item_col('test_item') \
+            .set_user_col('test_user') \
+            .set_output_col("item_score")
+
+        output = swing.transform(self.train_data.alias(['test_user', 'test_item']))[0]
+        self.assertEqual(
+            ['test_item', 'item_score'],
+            output.get_schema().get_field_names())
+
+    def test_transform(self):
+        swing = Swing().set_min_user_behavior(1)
+        output = swing.transform(self.train_data)[0]
+        self.verify_output_result(
+            output,
+            swing.get_item_col(),
+            output.get_schema().get_field_names(),
+            self.expected_data)
+
+    def test_save_load_and_transform(self):
+        swing = Swing().set_min_user_behavior(1)
+        reloaded_swing = self.save_and_reload(swing)
+        output = reloaded_swing.transform(self.train_data)[0]
+        self.verify_output_result(
+            output,
+            swing.get_item_col(),
+            output.get_schema().get_field_names(),
+            self.expected_data)
+
+    def test_data_type(self):
+        try:
+            swing = Swing().set_min_user_behavior(1)
+            output = swing.transform(self.wrong_type_data)[0]
+            self.t_env.to_data_stream(output).execute_and_collect()
+        except Py4JJavaError:
+            pass
+
+    def test_number_format(self):

Review Comment:
   Thanks for your advice. As you say, it cannot test if an exception was thrown. This test was attempt to check whether Error information is available and has been removed. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 merged pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 merged PR #192:
URL: https://github.com/apache/flink-ml/pull/192


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101090934


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
   Changed to:
   String itemList =
                           itemAndScore.stream()
                                   .sequential()
                                   .limit(itemNums)
                                   .map(tuple2 -> "" + tuple2.f0 + commaDelimiter + tuple2.f1)
                                   .collect(Collectors.joining("" + semicolonDelimiter));



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1103974792


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item columns must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testNumberFormat() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream = env.fromCollection(rows, trainDataTypeInfo);
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            Throwable exception = ExceptionUtils.getRootCause(e);
+            assertEquals(RuntimeException.class, exception.getClass());
+            assertEquals("Data of user and item column must not be null.", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+
+        assertEquals(
+                Arrays.asList("item_id", "item_score"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerItemCase() {

Review Comment:
   Let's also test `setMaxUserBehavior`. Could this be moved to `testTransform`?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053890313


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						userItemsMap.put(k, itemSet);
+					});
+				});
+
+			itemUsersMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"itemUsersMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+				.ifPresent(x -> {
+					itemUsersMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						itemUsersMap.put(k, itemSet);
+					});
+				});
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				tmpUerItemsMap.put(k, new ArrayList <>(v));
+			});
+			userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));

Review Comment:
   I am afraid it doesn't work. Type of `tmpUerItemsMap` is `Map <String, List <String>` and the type of `userItemsMap` is `Map <String, HashSet <String>>`.I am not sure could the type of `userItemsMapState` be `ListState <Map <String, HashSet <String>>>` ? This is the current definition of userItemsMapState.
   
   ```java
             userItemsMapState =
                   context.getOperatorStateStore()
                       .getListState(
                           new ListStateDescriptor <>(
                               "userItemsMapState",
                               Types.MAP(Types.STRING, Types.LIST(Types.STRING))));
   ```



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053957894


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),

Review Comment:
   I just pulled the code, and tried to modify and run it. 
   
   Actually, you met a `ClassCastException` here because `items` and `scores` in `SwingModelData` are defined as `ArrayList`. After defining them as `ArrayList`, an additional cast is used in `ModelDataDecoder#read`.
   
   At least, the cast in `ModelDataDecoder#read` is inappropriate, as usually you cannot assume more information than the API provides.
   For other places, using interfaces usually gives more flexibility than using concrete classes.
   
   



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on PR #192:
URL: https://github.com/apache/flink-ml/pull/192#issuecomment-1432938477

   Thanks for the update. LGTM.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
jiangxin369 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1065935317


##########
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());
+
+    default String getUserCol() {

Review Comment:
   Let's follow the existing code style to declare all the variables before declaring functions.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(

Review Comment:
   Would it be better to replace the `NumberFormatException ` with `RuntimeException`? Since we got null values and didn't format anything.



##########
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());
+
+    default String getUserCol() {
+        return get(USER_COL);
+    }
+
+    default T setUserCol(String value) {
+        return set(USER_COL, value);
+    }
+
+    Param<String> ITEM_COL =
+            new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull());
+
+    default String getItemCol() {
+        return get(ITEM_COL);
+    }
+
+    default T setItemCol(String value) {
+        return set(ITEM_COL, value);
+    }
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of related items for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    default int getK() {
+        return get(K);
+    }
+
+    default T setK(Integer value) {
+        return set(K, value);
+    }
+
+    Param<Integer> MAX_ITEM_USERS =

Review Comment:
   Let's rename it to `MAX_ITEM_NUMBER` to keep consistent with the name of `Param`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();

Review Comment:
   As @weibozhao commented, the casting is redundant, we can remove it.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 AlgoOperator<Swing>, SwingParams<Swing> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public Swing() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+

Review Comment:
   It would be better to add an argument checking here, `Preconditions.checkArgument(inputs.length == 1);`



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;

Review Comment:
   Why are these two variables not `private`?



##########
flink-ml-python/pyflink/ml/lib/recommendation/common.py:
##########
@@ -0,0 +1,48 @@
+################################################################################
+#  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 abc import ABC, abstractmethod
+
+from pyflink.ml.core.wrapper import JavaTransformer
+
+JAVA_RECOMMENDATION_PACKAGE_NAME = "org.apache.flink.ml.recommendation"
+
+
+class JavaRecommendationTransformer(JavaTransformer, ABC):
+    """
+    Wrapper class for a Java Regression Transformer.

Review Comment:
   Should it be `Java Recommendation Transformer`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }

Review Comment:
   We also need a `static T load(StreamTableEnvironment tEnv, String path)` function in each algorithm. You can refer to the doc of `interface Stage`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItems && !items.contains(item)) {

Review Comment:
   It's not efficient to operate `contains` on an `Arraylist`, how about replacing it with a data structure that is fast to index like `LinkedHashMap `?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItems && !items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;
+                for (int i = 0; i < itemNums; i++) {
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple3.of(
+                                            mainItem,
+                                            itemAndScore.get(i).f0,
+                                            itemAndScore.get(i).f1)));
+                }
+            }
+
+            userItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")

Review Comment:
   To distinguish from the `userItemsMapState` in `BuildSwingData`, we'd better give it a unique name.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append one column, that records all items the user has purchased, to the input table.

Review Comment:
   Per as the coding style of Flink,  comments often start in the third person, so let's replace the `Append` with `Appends`. So as the other comments.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItems && !items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;
+                for (int i = 0; i < itemNums; i++) {
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple3.of(
+                                            mainItem,
+                                            itemAndScore.get(i).f0,
+                                            itemAndScore.get(i).f1)));
+                }
+            }
+
+            userItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            userItemsMap.put(k, itemSet);
+                                        });
+                            });
+
+            itemUsersMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "itemUsersMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+                    .ifPresent(
+                            x -> {
+                                itemUsersMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            itemUsersMap.put(k, itemSet);
+                                        });
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        tmpUerItemsMap.put(k, new ArrayList<>(v));
+                    });
+            userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));
+
+            Map tmpItemUsersMap = new HashMap(itemUsersMap.size());
+            itemUsersMap.forEach(
+                    (k, v) -> {
+                        tmpItemUsersMap.put(k, new ArrayList<>(v));
+                    });
+            itemUsersMapState.update(Collections.singletonList(tmpItemUsersMap));

Review Comment:
   Why do you need to re-construct the `itemUsersMap` and save it to state? How about saving `itemUsersMap` directly?



##########
flink-ml-python/pyflink/examples/ml/recommendation/swing_example.py:
##########
@@ -0,0 +1,79 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment, Schema
+from pyflink.table.types import DataTypes
+
+from pyflink.ml.lib.recommendation.swing import Swing
+
+# create a new StreamExecutionEnvironment
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# create a StreamTableEnvironment
+t_env = StreamTableEnvironment.create(env)
+
+# generate input data

Review Comment:
   Please keep the same comment style in Java Example and change it to `Generates input data`. I understand that we have some python examples that not following the style currently, we'll refactor them in the future.



##########
flink-ml-examples/src/main/java/org/apache/flink/ml/examples/recommendation/SwingExample.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.recommendation;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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.flink.util.CloseableIterator;
+
+/** Simple program that trains a Swing model and uses it for recommendation. */
+public class SwingExample {
+    public static void main(String[] args) {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        // Generates input data.
+        DataStream<Row> inputStream =
+                env.fromElements(
+                        Row.of(new Long(null), 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));
+
+        Table inputTable = tEnv.fromDataStream(inputStream).as("user", "item");
+
+        // Creates a Swing object and initializes its parameters.
+        Swing swing = new Swing().setUserCol("user").setItemCol("item").setMinUserItems(1);
+
+        // Transform the data.

Review Comment:
   ```suggestion
           // Transforms the data.
   ```



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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, 11L, 0.058846),
+                            Row.of(10L, 12L, 0.058846),
+                            Row.of(11L, 10L, 0.058846),
+                            Row.of(11L, 12L, 0.058846),
+                            Row.of(12L, 10L, 0.058846),
+                            Row.of(12L, 11L, 0.058846),
+                            Row.of(12L, 13L, 0.091348),
+                            Row.of(13L, 12L, 0.091348)));
+
+    @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())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+    }
+
+    public void compareResultAndExpected(List<Row> results) {
+        results.sort(
+                new Comparator<Row>() {
+                    @Override
+                    public int compare(Row o1, Row o2) {
+                        long id1 = o1.getFieldAs(0);
+                        long id2 = o2.getFieldAs(0);
+                        if (id1 == id2) {
+                            id1 = ((Tuple2<Long, Double>) o1.getFieldAs(1)).f0;
+                            id2 = ((Tuple2<Long, Double>) o2.getFieldAs(1)).f0;
+                        }
+                        return (int) (id1 - id2);
+                    }
+                });
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            Tuple2<Long, Double> itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore.f0, expect.getField(1));
+            Assert.assertEquals(itemRankScore.f1, expect.getFieldAs(2), 1e-6);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(1000, swing.getMaxUserItems());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMinUserItems(10)
+                .setMaxUserItems(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(50, swing.getMaxUserItems());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+        DataStream<Row> dataStream = env.fromCollection(rows);
+        Schema schema =
+                Schema.newBuilder()
+                        .column("f0", DataTypes.INT())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        Table data = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+        Table[] swingResultTables =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserItems(1)
+                        .transform(data);
+
+        List<Row> results = IteratorUtils.toList(swingResultTables[0].execute().collect());

Review Comment:
   Please remove the unused variable.



##########
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());
+
+    default String getUserCol() {
+        return get(USER_COL);
+    }
+
+    default T setUserCol(String value) {
+        return set(USER_COL, value);
+    }
+
+    Param<String> ITEM_COL =
+            new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull());
+
+    default String getItemCol() {
+        return get(ITEM_COL);
+    }
+
+    default T setItemCol(String value) {
+        return set(ITEM_COL, value);
+    }
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of related items for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    default int getK() {
+        return get(K);
+    }
+
+    default T setK(Integer value) {
+        return set(K, value);
+    }
+
+    Param<Integer> MAX_ITEM_USERS =
+            new IntParam(
+                    "maxItemNumber",
+                    "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));
+
+    default int getMaxItemUsers() {
+        return get(MAX_ITEM_USERS);
+    }
+
+    default T setMaxItemUsers(Integer value) {
+        return set(MAX_ITEM_USERS, value);
+    }
+
+    Param<Integer> MIN_USER_ITEMS =
+            new IntParam(
+                    "minUserItems",
+                    "This parameter is used to exclude low-frequency users.",

Review Comment:
   The description is kind of not intuitive and users may not know exactly which value to set. Would it be better to change it to a descriptive statement like `The min number of items that...`? So as the other params.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column must be long.");

Review Comment:
   ```suggestion
               throw new IllegalArgumentException("The types of user and item columns must be Long.");
   ```



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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, 11L, 0.058846),
+                            Row.of(10L, 12L, 0.058846),
+                            Row.of(11L, 10L, 0.058846),
+                            Row.of(11L, 12L, 0.058846),
+                            Row.of(12L, 10L, 0.058846),
+                            Row.of(12L, 11L, 0.058846),
+                            Row.of(12L, 13L, 0.091348),
+                            Row.of(13L, 12L, 0.091348)));
+
+    @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())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+    }
+
+    public void compareResultAndExpected(List<Row> results) {
+        results.sort(
+                new Comparator<Row>() {
+                    @Override
+                    public int compare(Row o1, Row o2) {
+                        long id1 = o1.getFieldAs(0);
+                        long id2 = o2.getFieldAs(0);
+                        if (id1 == id2) {
+                            id1 = ((Tuple2<Long, Double>) o1.getFieldAs(1)).f0;
+                            id2 = ((Tuple2<Long, Double>) o2.getFieldAs(1)).f0;
+                        }
+                        return (int) (id1 - id2);
+                    }
+                });
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            Tuple2<Long, Double> itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore.f0, expect.getField(1));
+            Assert.assertEquals(itemRankScore.f1, expect.getFieldAs(2), 1e-6);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(1000, swing.getMaxUserItems());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMinUserItems(10)
+                .setMaxUserItems(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(50, swing.getMaxUserItems());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+        DataStream<Row> dataStream = env.fromCollection(rows);
+        Schema schema =
+                Schema.newBuilder()
+                        .column("f0", DataTypes.INT())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        Table data = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+        Table[] swingResultTables =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserItems(1)
+                        .transform(data);
+
+        List<Row> results = IteratorUtils.toList(swingResultTables[0].execute().collect());
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void testNumberFormat() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+        DataStream<Row> dataStream = env.fromCollection(rows);
+        Schema schema =
+                Schema.newBuilder()
+                        .column("f0", DataTypes.BIGINT())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        Table data = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserItems(1);
+        Table[] swingResultTables = swing.transform(data);
+        List<Row> results = IteratorUtils.toList(swingResultTables[0].execute().collect());

Review Comment:
   Please remove the unused variable.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests {@link Swing}. */
+public class SwingTest {

Review Comment:
   Please add test `testSaveLoadAndTransform` refer to `BucketizerTest`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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, 11L, 0.058846),
+                            Row.of(10L, 12L, 0.058846),
+                            Row.of(11L, 10L, 0.058846),
+                            Row.of(11L, 12L, 0.058846),
+                            Row.of(12L, 10L, 0.058846),
+                            Row.of(12L, 11L, 0.058846),
+                            Row.of(12L, 13L, 0.091348),
+                            Row.of(13L, 12L, 0.091348)));
+
+    @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())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+    }
+
+    public void compareResultAndExpected(List<Row> results) {

Review Comment:
   It should be `private`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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, 11L, 0.058846),
+                            Row.of(10L, 12L, 0.058846),
+                            Row.of(11L, 10L, 0.058846),
+                            Row.of(11L, 12L, 0.058846),
+                            Row.of(12L, 10L, 0.058846),
+                            Row.of(12L, 11L, 0.058846),
+                            Row.of(12L, 13L, 0.091348),
+                            Row.of(13L, 12L, 0.091348)));
+
+    @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())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+    }
+
+    public void compareResultAndExpected(List<Row> results) {
+        results.sort(
+                new Comparator<Row>() {
+                    @Override
+                    public int compare(Row o1, Row o2) {
+                        long id1 = o1.getFieldAs(0);
+                        long id2 = o2.getFieldAs(0);
+                        if (id1 == id2) {
+                            id1 = ((Tuple2<Long, Double>) o1.getFieldAs(1)).f0;
+                            id2 = ((Tuple2<Long, Double>) o2.getFieldAs(1)).f0;
+                        }
+                        return (int) (id1 - id2);
+                    }
+                });
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            Tuple2<Long, Double> itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore.f0, expect.getField(1));
+            Assert.assertEquals(itemRankScore.f1, expect.getFieldAs(2), 1e-6);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(1000, swing.getMaxUserItems());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMinUserItems(10)
+                .setMaxUserItems(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(50, swing.getMaxUserItems());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+        DataStream<Row> dataStream = env.fromCollection(rows);
+        Schema schema =
+                Schema.newBuilder()
+                        .column("f0", DataTypes.INT())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        Table data = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+        Table[] swingResultTables =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserItems(1)
+                        .transform(data);
+
+        List<Row> results = IteratorUtils.toList(swingResultTables[0].execute().collect());
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void testNumberFormat() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+        DataStream<Row> dataStream = env.fromCollection(rows);
+        Schema schema =
+                Schema.newBuilder()
+                        .column("f0", DataTypes.BIGINT())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        Table data = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserItems(1);
+        Table[] swingResultTables = swing.transform(data);
+        List<Row> results = IteratorUtils.toList(swingResultTables[0].execute().collect());
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserItems(1);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+
+        assertEquals(
+                Arrays.asList("item_id", "item_score"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerItemCase() {
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserItems(5);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+        List<Row> results = IteratorUtils.toList(output.execute().collect());
+        Assert.assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testFitAndPredict() throws Exception {

Review Comment:
   Swing doesn't fit and predict, it just transforms. Please rename the test function to `testTransform()`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator which implements the Swing algorithm.

Review Comment:
   `Swing` is not an Estimator, it is An AlgoOperator, so as the comment in Python.



##########
flink-ml-python/pyflink/ml/lib/recommendation/swing.py:
##########
@@ -0,0 +1,202 @@
+################################################################################
+#  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 pyflink.ml.core.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.core.wrapper import JavaWithParams
+from pyflink.ml.lib.param import HasOutputCol
+from pyflink.ml.lib.recommendation.common import JavaRecommendationTransformer
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "Name of user column.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Name of item column.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of related items for each item.",
+        10,

Review Comment:
   Let's set the default value to 100 to keep consistent with Java.



##########
flink-ml-python/pyflink/ml/lib/recommendation/swing.py:
##########
@@ -0,0 +1,202 @@
+################################################################################
+#  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 pyflink.ml.core.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.core.wrapper import JavaWithParams
+from pyflink.ml.lib.param import HasOutputCol
+from pyflink.ml.lib.recommendation.common import JavaRecommendationTransformer
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "Name of user column.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Name of item column.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of related items for each item.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_ITEM_USERS: Param[int] = IntParam(
+        "max_item_users",
+        "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))
+
+    MIN_USER_ITEMS: Param[int] = IntParam(
+        "min_user_items",
+        "This parameter is used to exclude low-frequency users.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_USER_ITEMS: Param[int] = IntParam(
+        "max_user_items",
+        "This parameter is used to exclude high-frequency users.",
+        1000,
+        ParamValidators.gt(0))
+
+    ALPHA1: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate weight of each user.",
+        15,
+        ParamValidators.gt_eq(0))
+
+    ALPHA2: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate similarity of users.",
+        0,
+        ParamValidators.gt_eq(0))
+
+    BETA: Param[float] = FloatParam(
+        "beta",
+        "This parameter is used to calculate weight of each user.",
+        0.3,
+        ParamValidators.gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_SwingParams, self).__init__(java_params)
+
+    def set_user_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.USER_COL, value))
+
+    def get_user_col(self) -> str:
+        return self.get(self.USER_COL)
+
+    def set_item_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.ITEM_COL, value))
+
+    def get_item_col(self) -> str:
+        return self.get(self.ITEM_COL)
+
+    def set_k(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.K, value))
+
+    def get_k(self) -> int:
+        return self.get(self.K)
+
+    def set_max_item_users(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_ITEM_USERS, value))
+
+    def get_max_item_users(self) -> int:
+        return self.get(self.MAX_ITEM_USERS)
+
+    def set_min_user_items(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MIN_USER_ITEMS, value))
+
+    def get_min_user_items(self) -> int:
+        return self.get(self.MIN_USER_ITEMS)
+
+    def set_max_user_items(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_ITEMS, value))
+
+    def get_max_user_items(self) -> int:
+        return self.get(self.MAX_USER_ITEMS)
+
+    def set_alpha1(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA1, value))
+
+    def get_alpha1(self) -> int:
+        return self.get(self.ALPHA1)
+
+    def set_alpha2(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA2, value))
+
+    def get_alpha2(self) -> int:
+        return self.get(self.ALPHA2)
+
+    def set_beta(self, value: float):
+        return typing.cast(_SwingParams, self.set(self.BETA, value))
+
+    def get_beta(self) -> float:
+        return self.get(self.BETA)
+
+    @property
+    def user_col(self) -> str:
+        return self.get_user_col()
+
+    @property
+    def item_col(self) -> str:
+        return self.get_item_col()
+
+    @property
+    def k(self) -> int:
+        return self.get_k()
+
+    @property
+    def max_item_users(self) -> int:
+        return self.get_max_item_users()
+
+    @property
+    def min_user_items(self) -> int:
+        return self.get_min_users_items()
+
+    @property
+    def max_user_items(self) -> int:
+        return self.get_max_user_items()
+
+    @property
+    def alpha1(self) -> int:
+        return self.get_alpha1()
+
+    @property
+    def alpha2(self) -> float:
+        return self.get_alpha2()
+
+    @property
+    def beta(self) -> float:
+        return self.get_beta()
+
+
+class Swing(JavaRecommendationTransformer, _SwingParams):

Review Comment:
   Please add tests for `swing.py`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItems && !items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(

Review Comment:
   We can replace the Comparator with lambda for better readability,
   ```
   itemAndScore.sort(
                           (o1, o2) -> 0 - Double.compare(o1.f1, o2.f1));
   ```



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "jiangxin369 (via GitHub)" <gi...@apache.org>.
jiangxin369 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1094573909


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItems && !items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;
+                for (int i = 0; i < itemNums; i++) {
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple3.of(
+                                            mainItem,
+                                            itemAndScore.get(i).f0,
+                                            itemAndScore.get(i).f1)));
+                }
+            }
+
+            userItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            userItemsMap.put(k, itemSet);
+                                        });
+                            });
+
+            itemUsersMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "itemUsersMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+                    .ifPresent(
+                            x -> {
+                                itemUsersMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            itemUsersMap.put(k, itemSet);
+                                        });
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        tmpUerItemsMap.put(k, new ArrayList<>(v));
+                    });
+            userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));
+
+            Map tmpItemUsersMap = new HashMap(itemUsersMap.size());
+            itemUsersMap.forEach(
+                    (k, v) -> {
+                        tmpItemUsersMap.put(k, new ArrayList<>(v));
+                    });
+            itemUsersMapState.update(Collections.singletonList(tmpItemUsersMap));

Review Comment:
   I see, thanks for the explanation.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1103973578


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher alpha1 is, the smaller weight each user gets.",
+                    15,
+                    ParamValidators.gtEq(0));
+
+    Param<Integer> ALPHA2 =
+            new IntParam(
+                    "alpha2",
+                    "This parameter is used to calculate similarity of users. "
+                            + "The higher alpha2 is, the less the similarity score is.",
+                    0,
+                    ParamValidators.gtEq(0));
+
+    Param<Double> BETA =
+            new DoubleParam(
+                    "beta",
+                    "This parameter is used to calculate weight of each user. "

Review Comment:
   How about the following description: 
   `Decay factor for number of users that have purchased one item. The higher beta is, the less purchasing behavior contributes to the similarity score.`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1106601429


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity between items in Swing is defined as
+ *
+ * <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. (<a
+ * href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String userCol = getUserCol();
+        final String itemCol = getItemCol();
+        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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    String.format("The maxUserBehavior must be larger or equal to minUserBehavior. "
+                                    + "The current value: maxUserBehavior=%d, minUserBehavior=%d.",
+                            getMaxUserBehavior(),
+                            getMinUserBehavior())
+            );
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> purchasingBehavior =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    Long userId = (Long) row.getFieldAs(userCol);
+                                    Long itemId = (Long) row.getFieldAs(itemCol);
+                                    if (userId == null || itemId == null) {
+                                        throw new RuntimeException(
+                                                "Data of user and item column must not be null.");
+                                    }
+                                    return Tuple2.of(userId, itemId);
+                                })
+                        .returns(Types.TUPLE(Types.LONG, Types.LONG));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Map<Long, String>>> userBehavior =
+                purchasingBehavior
+                        .keyBy(tuple -> tuple.f0)
+                        .transform(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                userBehavior
+                        .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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially, The appended column
+     * is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userAndPurchasedItems = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAndPurchasedItemsState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userAndPurchasedItems.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            items.forEach(
+                                    (item, nullValue) ->
+                                            output.collect(
+                                                    new StreamRecord<>(
+                                                            new Tuple3<>(user, item, items))));
+                        }
+                    });
+
+            userAndPurchasedItemsState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userAndPurchasedItems.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap<>();
+                userAndPurchasedItems.putIfAbsent(user, items);

Review Comment:
   This logic has been replaced by `getOrDefault` to make code a little clearer.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1106617016


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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",
+                    "User column name. The type of user column must be Long.",
+                    "user",
+                    ParamValidators.notNull());
+
+    Param<String> ITEM_COL =
+            new StringParam(
+                    "itemCol",
+                    "Item column name. The type of item column must be Long.",
+                    "item",
+                    ParamValidators.notNull());
+
+    Param<Integer> MAX_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "The max number of user(purchasers) for each item. If the number of user "
+                            + "is larger than this value, then only maxUserNumPerItem users will "
+                            + "be sampled and considered in the computation of similarity between two items.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item. If an item has "
+                            + "more than k recommendations, the first k similar items will be kept",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of items for a user purchases. If the items purchased by a user is smaller than "

Review Comment:
   Replaced by
   `This can affect the speed of the computation. Set maxUserBehavior smaller in case the swing recommendation progresses very slowly.`



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] weibozhao commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
weibozhao commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1063369892


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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 Table testData;
+    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, 11L, 0.058846),
+                            Row.of(10L, 12L, 0.058846),
+                            Row.of(11L, 10L, 0.058846),
+                            Row.of(11L, 12L, 0.058846),
+                            Row.of(12L, 10L, 0.058846),
+                            Row.of(12L, 11L, 0.058846),
+                            Row.of(12L, 13L, 0.091348),
+                            Row.of(13L, 12L, 0.091348)));
+
+    @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())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+    }
+
+    public void compareResultAndExpected(List<Row> results) {
+        results.sort(
+                new Comparator<Row>() {
+                    @Override
+                    public int compare(Row o1, Row o2) {
+                        long id1 = o1.getFieldAs(0);
+                        long id2 = o2.getFieldAs(0);
+                        if (id1 == id2) {
+                            id1 = ((Tuple2<Long, Double>) o1.getFieldAs(1)).f0;
+                            id2 = ((Tuple2<Long, Double>) o2.getFieldAs(1)).f0;
+                        }
+                        return (int) (id1 - id2);
+                    }
+                });
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            Tuple2<Long, Double> itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals((long) result.getFieldAs(0), (long) expect.getFieldAs(0));
+            Assert.assertEquals(itemRankScore.f0, (Long) expect.getFieldAs(1));
+            Assert.assertEquals(itemRankScore.f1, (Double) expect.getFieldAs(2), 1e-6);

Review Comment:
   These castings are not needed.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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 Table testData;
+    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, 11L, 0.058846),
+                            Row.of(10L, 12L, 0.058846),
+                            Row.of(11L, 10L, 0.058846),
+                            Row.of(11L, 12L, 0.058846),
+                            Row.of(12L, 10L, 0.058846),
+                            Row.of(12L, 11L, 0.058846),
+                            Row.of(12L, 13L, 0.091348),
+                            Row.of(13L, 12L, 0.091348)));
+
+    @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())
+                        .column("f1", DataTypes.BIGINT())
+                        .build();
+        DataStream<Row> dataStream = env.fromCollection(trainRows);
+        trainData = tEnv.fromDataStream(dataStream, schema).as("user_id", "item_id");
+    }
+
+    public void compareResultAndExpected(List<Row> results) {
+        results.sort(
+                new Comparator<Row>() {
+                    @Override
+                    public int compare(Row o1, Row o2) {
+                        long id1 = o1.getFieldAs(0);
+                        long id2 = o2.getFieldAs(0);
+                        if (id1 == id2) {
+                            id1 = ((Tuple2<Long, Double>) o1.getFieldAs(1)).f0;
+                            id2 = ((Tuple2<Long, Double>) o2.getFieldAs(1)).f0;
+                        }
+                        return (int) (id1 - id2);
+                    }
+                });
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            Tuple2<Long, Double> itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals((long) result.getFieldAs(0), (long) expect.getFieldAs(0));
+            Assert.assertEquals(itemRankScore.f0, (Long) expect.getFieldAs(1));
+            Assert.assertEquals(itemRankScore.f1, (Double) expect.getFieldAs(2), 1e-6);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(1000, swing.getMaxUserItems());
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMinUserItems(10)
+                .setMaxUserItems(50);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(10, swing.getMinUserItems());
+        assertEquals(50, swing.getMaxUserItems());
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Table input = trainData;

Review Comment:
   Please remove the no-used variable.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053913128


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());

Review Comment:
   Because `x` and `userItemsMap` are different types.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101144027


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
   nits: The `.sequential()` seems redundant, because `.stream()` already returns a sequential stream.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1100941086


##########
flink-ml-python/pyflink/ml/recommendation/swing.py:
##########
@@ -0,0 +1,209 @@
+################################################################################
+#  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 pyflink.ml.common.param import HasOutputCol
+from pyflink.ml.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.recommendation.common import JavaRecommendationTransformer
+from pyflink.ml.wrapper import JavaWithParams
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "Name of user column.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Name of item column.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of similar items to output for each item.",
+        100,
+        ParamValidators.gt(0))
+
+    MAX_USER_NUM_PER_ITEM: Param[int] = IntParam(
+        "max_user_num_per_item",
+        "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.",
+        1000,
+        ParamValidators.gt(0))
+
+    MIN_USER_BEHAVIOR: Param[int] = IntParam(
+        "min_user_behavior",
+        "The min number of interaction behavior between item and user.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_USER_BEHAVIOR: Param[int] = IntParam(
+        "max_user_behavior",
+        "The max number of interaction behavior between item and user."
+        + "The algorithm filters out activate users.",
+        1000,
+        ParamValidators.gt(0))
+
+    ALPHA1: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate weight of each user.",
+        15,
+        ParamValidators.gt_eq(0))
+
+    ALPHA2: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate similarity of users.",
+        0,
+        ParamValidators.gt_eq(0))
+
+    BETA: Param[float] = FloatParam(
+        "beta",
+        "This parameter is used to calculate weight of each user.",
+        0.3,
+        ParamValidators.gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_SwingParams, self).__init__(java_params)
+
+    def set_user_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.USER_COL, value))
+
+    def get_user_col(self) -> str:
+        return self.get(self.USER_COL)
+
+    def set_item_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.ITEM_COL, value))
+
+    def get_item_col(self) -> str:
+        return self.get(self.ITEM_COL)
+
+    def set_k(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.K, value))
+
+    def get_k(self) -> int:
+        return self.get(self.K)
+
+    def set_max_user_num_per_item(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_NUM_PER_ITEM, value))
+
+    def get_max_user_num_per_item(self) -> int:
+        return self.get(self.MAX_USER_NUM_PER_ITEM)
+
+    def set_min_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MIN_USER_BEHAVIOR, value))
+
+    def get_min_user_behavior(self) -> int:
+        return self.get(self.MIN_USER_BEHAVIOR)
+
+    def set_max_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_BEHAVIOR, value))
+
+    def get_max_user_behavior(self) -> int:
+        return self.get(self.MAX_USER_BEHAVIOR)
+
+    def set_alpha1(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA1, value))
+
+    def get_alpha1(self) -> int:
+        return self.get(self.ALPHA1)
+
+    def set_alpha2(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA2, value))
+
+    def get_alpha2(self) -> int:
+        return self.get(self.ALPHA2)
+
+    def set_beta(self, value: float):
+        return typing.cast(_SwingParams, self.set(self.BETA, value))
+
+    def get_beta(self) -> float:
+        return self.get(self.BETA)
+
+    @property
+    def user_col(self) -> str:
+        return self.get_user_col()
+
+    @property
+    def item_col(self) -> str:
+        return self.get_item_col()
+
+    @property
+    def k(self) -> int:
+        return self.get_k()
+
+    @property
+    def max_user_num_per_item(self) -> int:
+        return self.get_max_user_num_per_item()
+
+    @property
+    def min_user_behavior(self) -> int:
+        return self.get_min_user_behavior()
+
+    @property
+    def max_user_behavior(self) -> int:
+        return self.get_max_user_behavior()
+
+    @property
+    def alpha1(self) -> int:
+        return self.get_alpha1()
+
+    @property
+    def alpha2(self) -> float:
+        return self.get_alpha2()
+
+    @property
+    def beta(self) -> float:
+        return self.get_beta()
+
+
+class Swing(JavaRecommendationTransformer, _SwingParams):

Review Comment:
   In Java code, `Swing` is an implementation of `AlgoOperator`. So here `Swing` should extends corresponding Python class.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1106617016


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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",
+                    "User column name. The type of user column must be Long.",
+                    "user",
+                    ParamValidators.notNull());
+
+    Param<String> ITEM_COL =
+            new StringParam(
+                    "itemCol",
+                    "Item column name. The type of item column must be Long.",
+                    "item",
+                    ParamValidators.notNull());
+
+    Param<Integer> MAX_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "The max number of user(purchasers) for each item. If the number of user "
+                            + "is larger than this value, then only maxUserNumPerItem users will "
+                            + "be sampled and considered in the computation of similarity between two items.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item. If an item has "
+                            + "more than k recommendations, the first k similar items will be kept",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of items for a user purchases. If the items purchased by a user is smaller than "

Review Comment:
   Gave more info



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1108109143


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.TestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 org.apache.commons.lang3.exception.ExceptionUtils;
+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.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table inputTable;
+
+    @Before
+    public void before() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        List<Row> inputRows =
+                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),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 10L),
+                                Row.of(4L, 11L),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 13L)));
+        inputTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                inputRows,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            BasicTypeInfo.LONG_TYPE_INFO,
+                                            BasicTypeInfo.LONG_TYPE_INFO
+                                        },
+                                        new String[] {"user_id", "item_id"})));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        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")));
+
+        results.sort(Comparator.comparing(o -> o.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testInputWithIllegalDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testInputWithNull() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();

Review Comment:
   Replaced by `collect()`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053958500


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						userItemsMap.put(k, itemSet);
+					});
+				});
+
+			itemUsersMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"itemUsersMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+				.ifPresent(x -> {
+					itemUsersMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						itemUsersMap.put(k, itemSet);
+					});
+				});
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				tmpUerItemsMap.put(k, new ArrayList <>(v));
+			});
+			userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));

Review Comment:
   Sorry that I missed the types by the lengthy conversion codes 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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1090127528


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,444 @@
+/*
+ * 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.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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+
+        if (!(Types.LONG.equals(TableUtils.getTypeInfoByName(schema, userCol))
+                && Types.LONG.equals(TableUtils.getTypeInfoByName(schema, itemCol)))) {
+            throw new IllegalArgumentException("Type of user and item column 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 NumberFormatException(
+                                                "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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItems && !items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;
+                for (int i = 0; i < itemNums; i++) {
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple3.of(
+                                            mainItem,
+                                            itemAndScore.get(i).f0,
+                                            itemAndScore.get(i).f1)));
+                }
+            }
+
+            userItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            userItemsMap.put(k, itemSet);
+                                        });
+                            });
+
+            itemUsersMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "itemUsersMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+                    .ifPresent(
+                            x -> {
+                                itemUsersMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            itemUsersMap.put(k, itemSet);
+                                        });
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        tmpUerItemsMap.put(k, new ArrayList<>(v));
+                    });
+            userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));
+
+            Map tmpItemUsersMap = new HashMap(itemUsersMap.size());
+            itemUsersMap.forEach(
+                    (k, v) -> {
+                        tmpItemUsersMap.put(k, new ArrayList<>(v));
+                    });
+            itemUsersMapState.update(Collections.singletonList(tmpItemUsersMap));

Review Comment:
   Well, thanks for you advice. The problem I have is `itemUsersMap` cannot be used to update `itemUsersMapState`. The class of `itemUsersMap` is a Map<Long, HashSet<Long>> while `itemUsersMapState` is a ListState<Map<Long, List<Long>>>. So I need to construct a List<Long> object coming from  `itemUsersMap`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1099926295


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((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 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, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Set<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = new ArrayList<>(entry.getValue());
+                Long user = entry.getKey();
+                if (items.size() < minUserItemInteraction
+                        || items.size() > maxUserItemInteraction) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Set<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                Set<Long> value = new LinkedHashSet<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItemInteraction) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                for (long user : x.keySet()) {
+                                    List<Long> itemList = x.get(user);
+                                    userItemsMap.put(user, new LinkedHashSet<>(itemList));
+                                }
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map<Long, List<Long>> userItemsList = new HashMap<>(userItemsMap.size());
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                userItemsList.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+            }
+            userAllItemsMapState.update(Collections.singletonList(userItemsList));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, List<Long>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userLocalItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';

Review Comment:
   Have declared them as `final`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101153768


##########
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:
   Thanks for your explanation. Maybe we can add some explanation in the comments here, because it is a little confusing. 
   
   nits: BTW, Flink can support such usage by `MULTISET`. Although it is not better than `Map<Long, String>`, but it comes from official APIs.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1104201701


##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item columns must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testNumberFormat() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream = env.fromCollection(rows, trainDataTypeInfo);
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            Throwable exception = ExceptionUtils.getRootCause(e);
+            assertEquals(RuntimeException.class, exception.getClass());
+            assertEquals("Data of user and item column must not be null.", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+
+        assertEquals(
+                Arrays.asList("item_id", "item_score"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerItemCase() {

Review Comment:
   OK. Tests of `setMinUserBehavior` and `setMaxUserBehavior` have been moved to `testTransform`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101144027


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
   The `.sequential()` seems redundant, because `.stream()` already returns a sequential stream.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1093934595


##########
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:
   This operator is used to add the `outputTypeInfo` for `output` DataStream. It was silly of me to not have seen the second param of `transform` for passing output type and `map` has been deleted .



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "jiangxin369 (via GitHub)" <gi...@apache.org>.
jiangxin369 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1094650744


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((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 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, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Set<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = new ArrayList<>(entry.getValue());
+                Long user = entry.getKey();
+                if (items.size() < minUserItemInteraction
+                        || items.size() > maxUserItemInteraction) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Set<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                Set<Long> value = new LinkedHashSet<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItemInteraction) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                for (long user : x.keySet()) {
+                                    List<Long> itemList = x.get(user);
+                                    userItemsMap.put(user, new LinkedHashSet<>(itemList));
+                                }
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map<Long, List<Long>> userItemsList = new HashMap<>(userItemsMap.size());
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                userItemsList.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+            }
+            userAllItemsMapState.update(Collections.singletonList(userItemsList));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, List<Long>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userLocalItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userLocalItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userLocalItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userLocalItemsMapState, "userLocalItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            userItemsMap.put(k, itemSet);
+                                        });
+                            });
+
+            itemUsersMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "itemUsersMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+                    .ifPresent(
+                            x -> {
+                                itemUsersMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            itemUsersMap.put(k, itemSet);
+                                        });
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map tmpUerItemsMap = new HashMap(userItemsMap.size());

Review Comment:
   Let's replace the `Map` with `Map<Long, List<Long>>` to avoid raw use. So as the other parameterized class like `ArrayList`, `Map`. 
   
   Btw, if you are using Intellij, the editor would give some warnings for such cases, it is recommended to go through and fix the warnings.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101090934


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
    String itemList =
                           itemAndScore.stream()
                                   .sequential()
                                   .limit(itemNums)
                                   .map(tuple2 -> "" + tuple2.f0 + commaDelimiter + tuple2.f1)
                                   .collect(Collectors.joining("" + semicolonDelimiter));



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101144027


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
   The `.sequential()` seems redundant, because `.stream()` is already sequential.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101267515


##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static final Character commaDelimiter = ',';
+        private static final Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =

Review Comment:
   Thanks for your time, the reason I clone the set is that the modification in `interaction` won't effect on `userItemsMap`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1108207595


##########
docs/content/docs/operators/recommendation/swing.md:
##########
@@ -0,0 +1,194 @@
+---
+title: "Swing"
+type: docs
+aliases:
+- /operators/recommendation/swing.html
+---
+
+<!--
+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.
+-->
+
+## Swing
+
+An AlgoOperator which implements the Swing algorithm.
+
+Swing is an item recall algorithm. 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. 
+
+See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for Recommendation in
+E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.
+
+### Input Columns
+
+| Param name | Type | Default  | Description |
+|:-----------|:-----|:---------|:------------|
+| itemCol    | Long | `"item"` | Item id.    |
+| userCol    | Long | `"user"` | User id.    |
+### Output Columns
+
+| Param name | Type   | Default    | Description                                                                 |
+|:-----------|:-------|:-----------|:----------------------------------------------------------------------------|
+| itemCol    | Long   | `"item"`   | Item id.                                                                    |

Review Comment:
   nit: Let's align the rows here.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.PrimitiveArrayTypeInfo;
+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 org.apache.commons.lang3.ArrayUtils;
+
+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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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>Note that alpha1 and alpha2 could be zero here. If one of $$|I_u|, |I_v| and |I_u\cap I_v|$$
+ * is zero, then the similarity of <em>i</em> and <em>j</em> is zero.
+ *
+ * <p>See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for
+ * Recommendation in E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.
+ */
+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) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String userCol = getUserCol();
+        final String itemCol = getItemCol();
+        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 must be Long.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "The maxUserBehavior must be greater than or equal to minUserBehavior. "
+                                    + "The current setting: maxUserBehavior=%d, minUserBehavior=%d.",
+                            getMaxUserBehavior(), getMinUserBehavior()));
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> purchasingBehavior =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    Long userId = row.getFieldAs(userCol);
+                                    Long itemId = row.getFieldAs(itemCol);
+                                    if (userId == null || itemId == null) {
+                                        throw new RuntimeException(
+                                                "Data of user and item column must not be null.");
+                                    }
+                                    return Tuple2.of(userId, itemId);
+                                })
+                        .returns(Types.TUPLE(Types.LONG, Types.LONG));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, long[]>> userBehavior =
+                purchasingBehavior
+                        .keyBy(tuple -> tuple.f0)
+                        .transform(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        PrimitiveArrayTypeInfo.LONG_PRIMITIVE_ARRAY_TYPE_INFO),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                userBehavior
+                        .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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded.
+     */
+    private static class CollectingUserBehavior
+            extends AbstractStreamOperator<Tuple3<Long, Long, long[]>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, long[]>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userAndPurchasedItems` and
+        // `itemAndPurchasers` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userAndPurchasedItems = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAndPurchasedItemsState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userAndPurchasedItems.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            long[] itemsArray = new long[items.size()];
+                            int i = 0;
+                            for (Long value : items.keySet()) {
+                                itemsArray[i++] = value;
+                            }
+                            ArrayUtils.toPrimitive(items.keySet().toArray(new Long[0]));

Review Comment:
   This line seems useless. Could you remove it?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] weibozhao commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
weibozhao commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1063198634


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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());
+
+    default String getUserCol() {
+        return get(USER_COL);
+    }
+
+    default T setUserCol(String value) {
+        set(USER_COL, value);
+        return (T) this;
+    }
+
+    Param<String> ITEM_COL =
+            new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull());
+
+    default String getItemCol() {
+        return get(ITEM_COL);
+    }
+
+    default T setItemCol(String value) {
+        set(ITEM_COL, value);
+        return (T) this;

Review Comment:
   Do the same action as above comment.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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());
+
+    default String getUserCol() {
+        return get(USER_COL);
+    }
+
+    default T setUserCol(String value) {
+        set(USER_COL, value);
+        return (T) this;
+    }
+
+    Param<String> ITEM_COL =
+            new StringParam("itemCol", "Name of item column.", "item", ParamValidators.notNull());
+
+    default String getItemCol() {
+        return get(ITEM_COL);
+    }
+
+    default T setItemCol(String value) {
+        set(ITEM_COL, value);
+        return (T) this;
+    }
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of related items for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    default int getK() {
+        return get(K);
+    }
+
+    default T setK(Integer value) {
+        return set(K, value);
+    }
+
+    Param<Integer> MAX_ITEM_USERS =
+            new IntParam(
+                    "maxItemNumber",
+                    "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));
+
+    default int getMaxItemUsers() {
+        return get(MAX_ITEM_USERS);
+    }
+
+    default T setMaxItemUsers(Integer value) {

Review Comment:
   This API function is never used. It's better to test this Function in the ut. 
   Do the same things for  `setAlpha1(Integer value)`, `setAlpha2(Integer value)`, `setBeta(Double value)`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();

Review Comment:
   This casting is not needed. You can remove it.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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());
+
+    default String getUserCol() {
+        return get(USER_COL);
+    }
+
+    default T setUserCol(String value) {
+        set(USER_COL, value);
+        return (T) this;

Review Comment:
   You can use `return set(USER_COL, value);` to replace lines 43-44.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row ->
+                                        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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (!items.contains(item)) {
+                    items.add(item);

Review Comment:
   If the size of the items is larger than the `MAX_USER_ITEMS`, the adding action has no meaning. 
   In this case, `continue` is a better choice.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row ->
+                                        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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (!items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);

Review Comment:
   `0` can be removed here.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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;

Review Comment:
   `env, tEnv` can be converted to local variables.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row ->
+                                        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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (!items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;

Review Comment:
   `int itemNums = Math.min(k, itemAndScore.size());` maybe a better choice. 



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row ->
+                                        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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (!items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;
+                for (int i = 0; i < itemNums; i++) {
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple3.of(
+                                            mainItem,
+                                            itemAndScore.get(i).f0,
+                                            itemAndScore.get(i).f1)));
+                }
+            }
+
+            userItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int maxItemNumber) {
+            int n = allUsers.size();
+            if (n < maxItemNumber) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList(n);

Review Comment:
   I think `maxItemNumber` is more appropriate than `n`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row ->
+                                        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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                similarity.map(
+                        new MapFunction<Tuple3<Long, Long, Double>, Row>() {
+                            @Override
+                            public Row map(Tuple3<Long, Long, Double> value) throws Exception {
+                                Row row = new Row(2);
+                                row.setField(0, value.f0);
+                                row.setField(1, Tuple2.of(value.f1, value.f2));
+                                return row;
+                            }
+                        },
+                        outputTypeInfo);
+
+        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);
+    }
+
+    /**
+     * Append 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 BuildSwingData
+            extends AbstractStreamOperator<Tuple3<Long, Long, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        final int minUserItems;
+        final int maxUserItems;
+
+        private Map<Long, List<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+
+        private BuildSwingData(int minUserItems, int maxUserItems) {
+            this.minUserItems = minUserItems;
+            this.maxUserItems = maxUserItems;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, List<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = entry.getValue();
+                Long user = entry.getKey();
+                if (items.size() < minUserItems || items.size() > maxUserItems) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            List<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                List<Long> value = new ArrayList<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (!items.contains(item)) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+                    .ifPresent(x -> userItemsMap = x);
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculate top N similar items of each item. */
+    private static class CalculateSimilarity
+            extends AbstractStreamOperator<Tuple3<Long, Long, Double>>
+            implements OneInputStreamOperator<
+                            Tuple3<Long, Long, List<Long>>, Tuple3<Long, Long, Double>>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxItemUser;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private CalculateSimilarity(int k, int maxItemUser, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxItemUser = maxItemUser;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxItemUser);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort(
+                        new Comparator<Tuple2<Long, Double>>() {
+                            @Override
+                            public int compare(Tuple2<Long, Double> o1, Tuple2<Long, Double> o2) {
+                                return 0 - Double.compare(o1.f1, o2.f1);
+                            }
+                        });
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = k > itemAndScore.size() ? itemAndScore.size() : k;
+                for (int i = 0; i < itemNums; i++) {
+                    output.collect(
+                            new StreamRecord<>(
+                                    Tuple3.of(
+                                            mainItem,
+                                            itemAndScore.get(i).f0,
+                                            itemAndScore.get(i).f1)));
+                }
+            }
+
+            userItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int maxItemNumber) {
+            int n = allUsers.size();
+            if (n < maxItemNumber) {
+                return new ArrayList(allUsers);

Review Comment:
   Unchecked assignment: 'java.util.ArrayList' to 'java.util.List<java.lang.Long>' . `return new ArrayList<>(allUsers);` can fix this unchecked.
   
   You need to check this case in other palaces of your code.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row ->
+                                        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()));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Double>> similarity =
+                userAllItemsStream
+                        .keyBy(tuple -> tuple.f1)
+                        .transform(
+                                "calculateSwingSimilarity",
+                                Types.TUPLE(Types.LONG, Types.LONG, Types.DOUBLE),
+                                new CalculateSimilarity(
+                                        getK(),
+                                        getMaxItemUsers(),
+                                        getAlpha1(),
+                                        getAlpha2(),
+                                        getBeta()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {
+                            BasicTypeInfo.LONG_TYPE_INFO,
+                            Types.TUPLE(
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO)
+                        },

Review Comment:
   I think one output row need contains the item and the recommended info which includes recommended items and the scores.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.recommendation.swing.Swing;
+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.Comparator;
+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 Table testData;

Review Comment:
   The `testData` is never used, you can remove it.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,448 @@
+/*
+ * 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.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.AlgoOperator;
+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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * An Estimator 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 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();
+        final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+        final LogicalType userColType =
+                schema.getColumn(userCol).get().getDataType().getLogicalType();
+        final LogicalType itemColType =
+                schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+        if ((!TypeCheckUtils.isLong(InternalTypeInfo.of(userColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(userColType).toLogicalType()))
+                || (!TypeCheckUtils.isLong(InternalTypeInfo.of(itemColType).toLogicalType())
+                        && !TypeCheckUtils.isInteger(
+                                InternalTypeInfo.of(itemColType).toLogicalType()))) {
+            throw new IllegalArgumentException("Type of user and item column must be long or int.");

Review Comment:
   I think supporting the long type is enough. If you support the int type, what about string or timestamps?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053881373


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();

Review Comment:
   You're right. I agree to memorize and reuse them. But I think pre-computation of `similarity` for each user pair is impossible, especially when the matrix is huge and sparse. Should I use a fixed length array or sparse matrix to store high-frequency pairs? Will creating and searching this structure cost too much time?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();

Review Comment:
   Thanks for you  suggestion, I checked the code and am sure that a shallow copy can work as intended in this calculation. This copy is used to count number same of items in two `HashSet` instances, and has its own`map` that points to different `HashMap` instance. As for items in the `map`, this method doesn't modify value of items because it is used to count number of `interaction` so I think `clone` can work in this envionment.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053881499


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),

Review Comment:
   Thanks for reminding me. I met a `ClassCastException` when use `Arrays.asList(itemIds)` so used an inappropriate class transformation.
   > Caused by: java.lang.ClassCastException: java.util.Arrays$ArrayList cannot be cast to java.util.ArrayList
   	at org.apache.flink.ml.recommendation.swing.Swing$1.map(Swing.java:130)
   	
   I have modified type of `itemIds` and `itemScores` to ArrayList, you can check it.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1100202752


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((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 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, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Set<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = new ArrayList<>(entry.getValue());
+                Long user = entry.getKey();
+                if (items.size() < minUserItemInteraction
+                        || items.size() > maxUserItemInteraction) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Set<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                Set<Long> value = new LinkedHashSet<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItemInteraction) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                for (long user : x.keySet()) {
+                                    List<Long> itemList = x.get(user);
+                                    userItemsMap.put(user, new LinkedHashSet<>(itemList));
+                                }
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map<Long, List<Long>> userItemsList = new HashMap<>(userItemsMap.size());
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                userItemsList.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+            }
+            userAllItemsMapState.update(Collections.singletonList(userItemsList));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, List<Long>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userLocalItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userLocalItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userLocalItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userLocalItemsMapState, "userLocalItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            userItemsMap.put(k, itemSet);
+                                        });
+                            });
+
+            itemUsersMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "itemUsersMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+                    .ifPresent(
+                            x -> {
+                                itemUsersMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            itemUsersMap.put(k, itemSet);
+                                        });
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map tmpUerItemsMap = new HashMap(userItemsMap.size());

Review Comment:
   At your suggestion, I have removed `tmpUerItemsMap` and code for re-constructing values of `ListStat`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101267515


##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static final Character commaDelimiter = ',';
+        private static final Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =

Review Comment:
   Thanks for your time, the reason I clone the set is that the modification in `interaction` won't effect on `userItemsMap`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1103961097


##########
flink-ml-examples/src/main/java/org/apache/flink/ml/examples/recommendation/SwingExample.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.recommendation;
+
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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.flink.util.CloseableIterator;
+
+/** Simple program that creates a Swing instance and uses it to give recommendations for items. */

Review Comment:
   nit: ... and uses it to `generate` recommendations for items.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =

Review Comment:
   Is `purchasingBehavior` more explainable than `itemUsers`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");

Review Comment:
   Let's also print the value of `maxUserBehavior` and `minUserBehavior` in the error message to better debugging.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially,
+     * The appended column is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            items.forEach(
+                                    (item, nullValue) ->
+                                            output.collect(
+                                                    new StreamRecord<>(
+                                                            new Tuple3<>(user, item, items))));
+                        }
+                    });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);

Review Comment:
   Is `userAndPurchasedItems` more intuitive?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially,
+     * The appended column is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.forEach(

Review Comment:
   It seems that we should output a set instead of a map. Am I understanding it correctly?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "The max number of users that has purchased for each item. If the number of users that have "

Review Comment:
   Has --> Have



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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);

Review Comment:
   Let's move this check at the begining of method.
   
   In general, we want to make the error message to show up as early as possible.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher alpha1 is, the smaller weight each user gets.",
+                    15,
+                    ParamValidators.gtEq(0));
+
+    Param<Integer> ALPHA2 =
+            new IntParam(
+                    "alpha2",
+                    "This parameter is used to calculate similarity of users. "
+                            + "The higher alpha2 is, the less the similarity score is.",
+                    0,
+                    ParamValidators.gtEq(0));
+
+    Param<Double> BETA =
+            new DoubleParam(
+                    "beta",
+                    "This parameter is used to calculate weight of each user. "

Review Comment:
   Decay factor for number of users that have purchased one item. The higher beta is, the less purchasing behavior contributes to the similarity score.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially,
+     * The appended column is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            items.forEach(
+                                    (item, nullValue) ->
+                                            output.collect(
+                                                    new StreamRecord<>(
+                                                            new Tuple3<>(user, item, items))));
+                        }
+                    });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap<>();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates similarity between items and keep top k similar items of each target item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();

Review Comment:
   Are `userAndPurchasedItems` and `itemAndPurchasedUsers` more intutive?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher alpha1 is, the smaller weight each user gets.",
+                    15,
+                    ParamValidators.gtEq(0));
+
+    Param<Integer> ALPHA2 =
+            new IntParam(
+                    "alpha2",
+                    "This parameter is used to calculate similarity of users. "
+                            + "The higher alpha2 is, the less the similarity score is.",
+                    0,

Review Comment:
   `alpha2` should be greater than zero, right?



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item columns must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testNumberFormat() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream = env.fromCollection(rows, trainDataTypeInfo);
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            Throwable exception = ExceptionUtils.getRootCause(e);
+            assertEquals(RuntimeException.class, exception.getClass());
+            assertEquals("Data of user and item column must not be null.", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+
+        assertEquals(
+                Arrays.asList("item_id", "item_score"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerItemCase() {

Review Comment:
   Let's also test `setMaxUserBehavior`. Could this move to `testTransform`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> itemUsers =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    if (row.getFieldAs(userCol) == null

Review Comment:
   We should probably avoid repeatedly accessing a field by name since it is not that efficient.
   
   Can you store it in a local variable and use it in the return statement?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially,
+     * The appended column is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();

Review Comment:
   Is `userAndPurchasedItems` more intuitive?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially,
+     * The appended column is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            items.forEach(
+                                    (item, nullValue) ->
+                                            output.collect(
+                                                    new StreamRecord<>(
+                                                            new Tuple3<>(user, item, items))));
+                        }
+                    });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap<>();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);

Review Comment:
   It seems that we should remove this line and put the items in Line#220.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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 =

Review Comment:
   Is `userBehavior` easier for understanding the semantic of the computation logic?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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());

Review Comment:
   Let's update the description as `User column name` and `User column name` following the existing conventions like `HasFeaturesCol`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",

Review Comment:
   How about using the following description:
   `The min number of items that a user purchases. If the items purchased by a user is smaller than this value, then this user is filtered out and will not be used in the computation logic.`



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "

Review Comment:
   How about using the following description:
   `The max number of items that a user purchases. If the items purchased by a user is larger than this value, then this user is filtered out and will not be used in the computation logic.`



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =

Review Comment:
   Let's make `trainDataTypeInfo` and `trainRows` local variables since they are used only once.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item columns must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testNumberFormat() {

Review Comment:
   How about renaming it as `testInputWithNull`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher alpha1 is, the smaller weight each user gets.",
+                    15,
+                    ParamValidators.gtEq(0));
+
+    Param<Integer> ALPHA2 =
+            new IntParam(
+                    "alpha2",
+                    "This parameter is used to calculate similarity of users. "

Review Comment:
   Smooth factor for number of users that have purchased the two target items. The higher alpha2 is, the less purchasing behavior contributes to the similarity score.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher alpha1 is, the smaller weight each user gets.",
+                    15,

Review Comment:
   As discussed offline, we should set the default values properly. Could you double-check the default values of these parameters?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "

Review Comment:
   `Smooth factor for number of users that have purchased one item. The higher alpha1 is, the less purchasing behavior contributes to the similarity score.`



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {

Review Comment:
   Let's make it a more informative name, e.g., 'testInputWithIllegalDataType'.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1099929243


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());

Review Comment:
   Yep



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1100094424


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();

Review Comment:
   We can use `String#join` or similar methods to make these lines concise, and no need to copy whole string with removing last character.
   ```
   String itemList =
                           itemAndScore.stream()
                                   .map(d -> "" + d.f0 + commaDelimiter + d.f1)
                                   .collect(Collectors.joining("" + semicolonDelimiter));
   ```



##########
flink-ml-python/pyflink/ml/lib/recommendation/swing.py:
##########
@@ -0,0 +1,209 @@
+################################################################################
+#  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 pyflink.ml.core.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.core.wrapper import JavaWithParams
+from pyflink.ml.lib.param import HasOutputCol
+from pyflink.ml.lib.recommendation.common import JavaRecommendationTransformer
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "Name of user column.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Name of item column.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of similar items to output for each item.",
+        100,
+        ParamValidators.gt(0))
+
+    MAX_USER_NUM_PER_ITEM: Param[int] = IntParam(
+        "max_user_num_per_item",
+        "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.",
+        1000,
+        ParamValidators.gt(0))
+
+    MIN_USER_BEHAVIOR: Param[int] = IntParam(
+        "min_user_behavior",
+        "The min number of interaction behavior between item and user.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_USER_BEHAVIOR: Param[int] = IntParam(
+        "max_user_behavior",
+        "The max number of interaction behavior between item and user."
+        + "The algorithm filters out activate users.",
+        1000,
+        ParamValidators.gt(0))
+
+    ALPHA1: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate weight of each user.",
+        15,
+        ParamValidators.gt_eq(0))
+
+    ALPHA2: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate similarity of users.",
+        0,
+        ParamValidators.gt_eq(0))
+
+    BETA: Param[float] = FloatParam(
+        "beta",
+        "This parameter is used to calculate weight of each user.",
+        0.3,
+        ParamValidators.gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_SwingParams, self).__init__(java_params)
+
+    def set_user_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.USER_COL, value))
+
+    def get_user_col(self) -> str:
+        return self.get(self.USER_COL)
+
+    def set_item_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.ITEM_COL, value))
+
+    def get_item_col(self) -> str:
+        return self.get(self.ITEM_COL)
+
+    def set_k(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.K, value))
+
+    def get_k(self) -> int:
+        return self.get(self.K)
+
+    def set_max_user_num_per_item(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_NUM_PER_ITEM, value))
+
+    def get_max_user_num_per_item(self) -> int:
+        return self.get(self.MAX_USER_NUM_PER_ITEM)
+
+    def set_min_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MIN_USER_BEHAVIOR, value))
+
+    def get_min_user_behavior(self) -> int:
+        return self.get(self.MIN_USER_BEHAVIOR)
+
+    def set_max_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_BEHAVIOR, value))
+
+    def get_max_user_behavior(self) -> int:
+        return self.get(self.MAX_USER_BEHAVIOR)
+
+    def set_alpha1(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA1, value))
+
+    def get_alpha1(self) -> int:
+        return self.get(self.ALPHA1)
+
+    def set_alpha2(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA2, value))
+
+    def get_alpha2(self) -> int:
+        return self.get(self.ALPHA2)
+
+    def set_beta(self, value: float):
+        return typing.cast(_SwingParams, self.set(self.BETA, value))
+
+    def get_beta(self) -> float:
+        return self.get(self.BETA)
+
+    @property
+    def user_col(self) -> str:
+        return self.get_user_col()
+
+    @property
+    def item_col(self) -> str:
+        return self.get_item_col()
+
+    @property
+    def k(self) -> int:
+        return self.get_k()
+
+    @property
+    def max_user_num_per_item(self) -> int:
+        return self.get_max_user_num_per_item()
+
+    @property
+    def min_user_behavior(self) -> int:
+        return self.get_min_user_behavior()
+
+    @property
+    def max_user_behavior(self) -> int:
+        return self.get_max_user_behavior()
+
+    @property
+    def alpha1(self) -> int:
+        return self.get_alpha1()
+
+    @property
+    def alpha2(self) -> float:
+        return self.get_alpha2()
+
+    @property
+    def beta(self) -> float:
+        return self.get_beta()
+
+
+class Swing(JavaRecommendationTransformer, _SwingParams):

Review Comment:
   In Java code, Swing is an implementation of `AlgoOperator`. So I guess here in Python code, Swing should extends `JavaRecommendationAlgoOperator`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        DataStream<Row> dataStream = env.fromCollection(trainRows, trainDataTypeInfo);
+        trainData = tEnv.fromDataStream(dataStream);
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));

Review Comment:
   The expected value should be the first parameter according to its parameter names. So as other places.



##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()

Review Comment:
   Let's simplify `...stream().forEach(...)` to `...forEach(...)`, if `stream()` is not necessary.  IntelliJ also gives further suggestions by using `userItemsMap.forEach` to simplify this part. So as `items.entrySet().stream()` below.



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

Review Comment:
   Since we have just used `Types.LONG` and `Types.STRING`, let's continue using them here to make it consistent.



##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static final Character commaDelimiter = ',';
+        private static final Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =

Review Comment:
   Let's use `Set<Long> interaction = new HashSet<>(userItemsMap.get(u).keySet());` It provides same function with cleaner code.



##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();

Review Comment:
   The same issue as `userItemsMap` in `CollectingUserBehavior`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore, expect.getField(1));
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {
+        String msg = "";
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            msg = e.getMessage();
+            assertEquals("The types of user and item columns must be Long.", msg);
+        }
+    }
+
+    @Test
+    public void testNumberFormat() {
+        String msg = "";

Review Comment:
   Let's move this declaration to `catch` block, i.e., `String msg = e.getCause()...`. Same as other places.



##########
flink-ml-python/pyflink/ml/recommendation/swing.py:
##########
@@ -0,0 +1,209 @@
+################################################################################
+#  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 pyflink.ml.common.param import HasOutputCol
+from pyflink.ml.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.recommendation.common import JavaRecommendationTransformer
+from pyflink.ml.wrapper import JavaWithParams
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "Name of user column.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Name of item column.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of similar items to output for each item.",
+        100,
+        ParamValidators.gt(0))
+
+    MAX_USER_NUM_PER_ITEM: Param[int] = IntParam(
+        "max_user_num_per_item",
+        "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.",
+        1000,
+        ParamValidators.gt(0))
+
+    MIN_USER_BEHAVIOR: Param[int] = IntParam(
+        "min_user_behavior",
+        "The min number of interaction behavior between item and user.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_USER_BEHAVIOR: Param[int] = IntParam(
+        "max_user_behavior",
+        "The max number of interaction behavior between item and user."
+        + "The algorithm filters out activate users.",
+        1000,
+        ParamValidators.gt(0))
+
+    ALPHA1: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate weight of each user.",
+        15,
+        ParamValidators.gt_eq(0))
+
+    ALPHA2: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate similarity of users.",
+        0,
+        ParamValidators.gt_eq(0))
+
+    BETA: Param[float] = FloatParam(
+        "beta",
+        "This parameter is used to calculate weight of each user.",
+        0.3,
+        ParamValidators.gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_SwingParams, self).__init__(java_params)
+
+    def set_user_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.USER_COL, value))
+
+    def get_user_col(self) -> str:
+        return self.get(self.USER_COL)
+
+    def set_item_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.ITEM_COL, value))
+
+    def get_item_col(self) -> str:
+        return self.get(self.ITEM_COL)
+
+    def set_k(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.K, value))
+
+    def get_k(self) -> int:
+        return self.get(self.K)
+
+    def set_max_user_num_per_item(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_NUM_PER_ITEM, value))
+
+    def get_max_user_num_per_item(self) -> int:
+        return self.get(self.MAX_USER_NUM_PER_ITEM)
+
+    def set_min_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MIN_USER_BEHAVIOR, value))
+
+    def get_min_user_behavior(self) -> int:
+        return self.get(self.MIN_USER_BEHAVIOR)
+
+    def set_max_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_BEHAVIOR, value))
+
+    def get_max_user_behavior(self) -> int:
+        return self.get(self.MAX_USER_BEHAVIOR)
+
+    def set_alpha1(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA1, value))
+
+    def get_alpha1(self) -> int:
+        return self.get(self.ALPHA1)
+
+    def set_alpha2(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA2, value))
+
+    def get_alpha2(self) -> int:
+        return self.get(self.ALPHA2)
+
+    def set_beta(self, value: float):
+        return typing.cast(_SwingParams, self.set(self.BETA, value))
+
+    def get_beta(self) -> float:
+        return self.get(self.BETA)
+
+    @property
+    def user_col(self) -> str:
+        return self.get_user_col()
+
+    @property
+    def item_col(self) -> str:
+        return self.get_item_col()
+
+    @property
+    def k(self) -> int:
+        return self.get_k()
+
+    @property
+    def max_user_num_per_item(self) -> int:
+        return self.get_max_user_num_per_item()
+
+    @property
+    def min_user_behavior(self) -> int:
+        return self.get_min_user_behavior()
+
+    @property
+    def max_user_behavior(self) -> int:
+        return self.get_max_user_behavior()
+
+    @property
+    def alpha1(self) -> int:
+        return self.get_alpha1()
+
+    @property
+    def alpha2(self) -> float:
+        return self.get_alpha2()
+
+    @property
+    def beta(self) -> float:
+        return self.get_beta()
+
+
+class Swing(JavaRecommendationTransformer, _SwingParams):

Review Comment:
   In Java code, `Swing` is an implementation of `AlgoOperator`. So here `Swing ` should extends corresponding Python class.



##########
flink-ml-python/pyflink/ml/recommendation/tests/test_swing.py:
##########
@@ -0,0 +1,149 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.table import Table
+from typing import List
+from py4j.protocol import Py4JJavaError:
+
+from pyflink.ml.recommendation.swing import Swing
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+
+# Tests Swing. 
+class SwingTest(PyFlinkMLTestCase):
+    def setUp(self):
+        super(SwingTest, self).setUp()
+        self.train_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (0, 11),
+                (0, 12),
+                (1, 13),
+                (1, 12),
+                (2, 10),
+                (2, 11),
+                (2, 12),
+                (3, 13),
+                (3, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.wrong_type_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (1, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.INT(), Types.LONG()])
+            ))
+
+        self.none_value_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (None, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.expected_data = [
+            [10, '11,0.058845768947156235;12,0.058845768947156235'],
+            [11, '10,0.058845768947156235;12,0.058845768947156235'],
+            [12, '13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235'],
+            [13, '12,0.09134833828228624']]
+
+    def test_param(self):
+        swing = Swing()
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(100, swing.k)
+        self.assertEqual(10, swing.min_user_items)
+        self.assertEqual(1000, swing.max_user_items)
+        self.assertEqual(15, swing.alpha1)
+        self.assertEqual(0, swing.alpha2)
+        self.assertEqual(0.3, swing.beta, delta=1e-9)
+
+        swing.set_item_col("item_1") \
+            .set_user_col("user_1") \
+            .set_k(20) \
+            .set_min_user_behavior(20) \
+            .set_max_user_behavior(50) \
+            .set_alpha1(5) \
+            .set_alpha2(1) \
+            .set_beta(0.35)
+
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(20, swing.k)
+        self.assertEqual(20, swing.min_user_items)
+        self.assertEqual(50, swing.max_user_items)
+        self.assertEqual(5, swing.alpha1)
+        self.assertEqual(1, swing.alpha2)
+        self.assertEqual(0.35, swing.beta, delta=1e-9)

Review Comment:
   In Python, we should use `assertAlmostEqual`. The IDE should give warning about this line.



##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Map<Long, String> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers.keySet());
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();

Review Comment:
   Use `Random` without seed will make the results unreproducible. Is it possible to use deterministic seeds here?



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

Review Comment:
   It is vague to say `formula of Swing`. It's better to change `The formula of Swing is` to `The similarity between items in Swing is defined as`.  So as in Python document.



##########
flink-ml-python/pyflink/ml/lib/recommendation/swing.py:
##########
@@ -0,0 +1,209 @@
+################################################################################
+#  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 pyflink.ml.core.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.core.wrapper import JavaWithParams
+from pyflink.ml.lib.param import HasOutputCol
+from pyflink.ml.lib.recommendation.common import JavaRecommendationTransformer
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "Name of user column.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Name of item column.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of similar items to output for each item.",
+        100,
+        ParamValidators.gt(0))
+
+    MAX_USER_NUM_PER_ITEM: Param[int] = IntParam(
+        "max_user_num_per_item",
+        "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.",
+        1000,
+        ParamValidators.gt(0))
+
+    MIN_USER_BEHAVIOR: Param[int] = IntParam(
+        "min_user_behavior",
+        "The min number of interaction behavior between item and user.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_USER_BEHAVIOR: Param[int] = IntParam(
+        "max_user_behavior",
+        "The max number of interaction behavior between item and user."
+        + "The algorithm filters out activate users.",
+        1000,
+        ParamValidators.gt(0))
+
+    ALPHA1: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate weight of each user.",
+        15,
+        ParamValidators.gt_eq(0))
+
+    ALPHA2: Param[int] = IntParam(
+        "alpha2",
+        "This parameter is used to calculate similarity of users.",
+        0,
+        ParamValidators.gt_eq(0))
+
+    BETA: Param[float] = FloatParam(
+        "beta",
+        "This parameter is used to calculate weight of each user.",
+        0.3,
+        ParamValidators.gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_SwingParams, self).__init__(java_params)
+
+    def set_user_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.USER_COL, value))
+
+    def get_user_col(self) -> str:
+        return self.get(self.USER_COL)
+
+    def set_item_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.ITEM_COL, value))
+
+    def get_item_col(self) -> str:
+        return self.get(self.ITEM_COL)
+
+    def set_k(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.K, value))
+
+    def get_k(self) -> int:
+        return self.get(self.K)
+
+    def set_max_user_num_per_item(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_NUM_PER_ITEM, value))
+
+    def get_max_user_num_per_item(self) -> int:
+        return self.get(self.MAX_USER_NUM_PER_ITEM)
+
+    def set_min_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MIN_USER_BEHAVIOR, value))
+
+    def get_min_user_behavior(self) -> int:
+        return self.get(self.MIN_USER_BEHAVIOR)
+
+    def set_max_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_BEHAVIOR, value))
+
+    def get_max_user_behavior(self) -> int:
+        return self.get(self.MAX_USER_BEHAVIOR)
+
+    def set_alpha1(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA1, value))
+
+    def get_alpha1(self) -> int:
+        return self.get(self.ALPHA1)
+
+    def set_alpha2(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA2, value))
+
+    def get_alpha2(self) -> int:
+        return self.get(self.ALPHA2)
+
+    def set_beta(self, value: float):
+        return typing.cast(_SwingParams, self.set(self.BETA, value))
+
+    def get_beta(self) -> float:
+        return self.get(self.BETA)
+
+    @property
+    def user_col(self) -> str:
+        return self.get_user_col()
+
+    @property
+    def item_col(self) -> str:
+        return self.get_item_col()
+
+    @property
+    def k(self) -> int:
+        return self.get_k()
+
+    @property
+    def max_user_num_per_item(self) -> int:
+        return self.get_max_user_num_per_item()
+
+    @property
+    def min_user_behavior(self) -> int:
+        return self.get_min_user_behavior()
+
+    @property
+    def max_user_behavior(self) -> int:
+        return self.get_max_user_behavior()
+
+    @property
+    def alpha1(self) -> int:
+        return self.get_alpha1()
+
+    @property
+    def alpha2(self) -> float:
+        return self.get_alpha2()
+
+    @property
+    def beta(self) -> float:
+        return self.get_beta()
+
+
+class Swing(JavaRecommendationTransformer, _SwingParams):
+    """
+    An AlgoOperator which implements the Swing algorithm.
+    
+    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 u
+    and user v have purchased the same commodity i , they will form a relationship
+    diagram similar to a swing. If u and v have purchased commodity j in
+    addition to i, it is supposed i and j are similar. The formula of Swing is
+    $$ 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|}} $$

Review Comment:
   `\alpha\_2` should be `\alpha_2`, so as in Java document.



##########
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);

Review Comment:
   Let's have a check between `minUserBehavior` and `maxUserBehavior`  here, in case `minUserBehavior` is larger than `maxUserBehavior`.



##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();

Review Comment:
   ```suggestion
                   items = new LinkedHashMap<>();
   ```



##########
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:
   I'm confused about its type. In `endInput`, values of inner maps are never used. In `processElement`, the values of inner maps are always `null` (set by `items.put(item, null);`). Then, can we change its type to `Map<Long, Set<Long>>`?



##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static final Character commaDelimiter = ',';
+        private static final Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Map<Long, String> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers.keySet());
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers.keySet()) {

Review Comment:
   Current sampling may not accurately return `sampleSize` elements. 
   It seems not necessary to use streaming sampling here. Can we use `Collections.shuffle` then return first `sampleSize ` elements?



##########
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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static final Character commaDelimiter = ',';
+        private static final Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Map<Long, String> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers.keySet());

Review Comment:
   ```suggestion
                   return new ArrayList<>(allUsers.keySet());
   ```



##########
flink-ml-python/pyflink/examples/ml/recommendation/swing_example.py:
##########
@@ -0,0 +1,72 @@
+################################################################################
+#  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.
+################################################################################
+
+# Simple program that creates a Swing instance and gives recommendations for items.
+
+from pyflink.common import Types
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+from pyflink.ml.recommendation.swing import Swing
+
+# Creates a new StreamExecutionEnvironment.
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# Creates a StreamTableEnvironment.
+t_env = StreamTableEnvironment.create(env)
+
+# Generates input data.
+input_table = t_env.from_data_stream(
+    env.from_collection([
+        (0, 10),
+        (0, 11),
+        (0, 12),
+        (1, 13),
+        (1, 12),
+        (2, 10),
+        (2, 11),
+        (2, 12),
+        (3, 13),
+        (3, 12)
+    ],
+        type_info=Types.ROW_NAMED(
+        ['user', 'item'],
+        [Types.LONG(), Types.LONG()])
+    ))
+
+# Creates a swing object and initialize its parameters.
+swing = Swing()\
+    .set_item_col('item')\
+    .set_user_col("user")\
+    .set_min_user_behavior(1)
+
+# Transforms the data to Swing algorithm result.
+swingTable = swing.transform(input_table)

Review Comment:
   `swingTable` is ambiguous and also not conform to Python naming convention. Let's use `output` instead like other tests.



##########
flink-ml-python/pyflink/examples/ml/recommendation/swing_example.py:
##########
@@ -0,0 +1,72 @@
+################################################################################
+#  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.
+################################################################################
+
+# Simple program that creates a Swing instance and gives recommendations for items.
+
+from pyflink.common import Types
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment
+
+from pyflink.ml.recommendation.swing import Swing
+
+# Creates a new StreamExecutionEnvironment.
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# Creates a StreamTableEnvironment.
+t_env = StreamTableEnvironment.create(env)
+
+# Generates input data.
+input_table = t_env.from_data_stream(
+    env.from_collection([
+        (0, 10),
+        (0, 11),
+        (0, 12),
+        (1, 13),
+        (1, 12),
+        (2, 10),
+        (2, 11),
+        (2, 12),
+        (3, 13),
+        (3, 12)
+    ],
+        type_info=Types.ROW_NAMED(
+        ['user', 'item'],
+        [Types.LONG(), Types.LONG()])
+    ))
+
+# Creates a swing object and initialize its parameters.
+swing = Swing()\
+    .set_item_col('item')\
+    .set_user_col("user")\
+    .set_min_user_behavior(1)
+
+# Transforms the data to Swing algorithm result.
+swingTable = swing.transform(input_table)
+
+# Extracts and display the results.
+field_names = swingTable[0].get_schema().get_field_names()
+
+results = t_env.to_changelog_stream(

Review Comment:
   Let's `to_data_stream` here like other tests, if `to_changelog_stream` is not necessary.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 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;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        trainData = tEnv.fromDataStream(env.fromCollection(trainRows, trainDataTypeInfo));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore, expect.getField(1));
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testDataType() {
+        String msg = "";
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            msg = e.getMessage();
+            assertEquals("The types of user and item columns must be Long.", msg);
+        }
+    }
+
+    @Test
+    public void testNumberFormat() {
+        String msg = "";
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream = env.fromCollection(rows, trainDataTypeInfo);
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            msg = e.getCause().getCause().getCause().getCause().getCause().getMessage();

Review Comment:
   We can replace this line with `ExceptionUtils.getRootCause`. You can check its usage in other tests.



##########
flink-ml-python/pyflink/ml/recommendation/__init__.py:
##########
@@ -0,0 +1,17 @@
+################################################################################
+#  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.
+################################################################################

Review Comment:
   There should be a newline here.  
   
   There are some other code style problems. You can check your code style and types locally. Please refer to related commands here: [1]
   
   [1] [      - name: Check code style](https://github.com/apache/flink-ml/blob/76d5b864dd7df95adebebe6f7e47d52ae0f689c7/.github/workflows/python-tests.yml#L54)



##########
flink-ml-python/pyflink/ml/recommendation/tests/test_swing.py:
##########
@@ -0,0 +1,149 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.table import Table
+from typing import List
+from py4j.protocol import Py4JJavaError:
+
+from pyflink.ml.recommendation.swing import Swing
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+
+# Tests Swing. 
+class SwingTest(PyFlinkMLTestCase):
+    def setUp(self):
+        super(SwingTest, self).setUp()
+        self.train_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (0, 11),
+                (0, 12),
+                (1, 13),
+                (1, 12),
+                (2, 10),
+                (2, 11),
+                (2, 12),
+                (3, 13),
+                (3, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.wrong_type_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (1, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.INT(), Types.LONG()])
+            ))
+
+        self.none_value_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (None, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.expected_data = [
+            [10, '11,0.058845768947156235;12,0.058845768947156235'],
+            [11, '10,0.058845768947156235;12,0.058845768947156235'],
+            [12, '13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235'],
+            [13, '12,0.09134833828228624']]
+
+    def test_param(self):
+        swing = Swing()
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(100, swing.k)
+        self.assertEqual(10, swing.min_user_items)
+        self.assertEqual(1000, swing.max_user_items)
+        self.assertEqual(15, swing.alpha1)
+        self.assertEqual(0, swing.alpha2)
+        self.assertEqual(0.3, swing.beta, delta=1e-9)

Review Comment:
   In Python, we should use `assertAlmostEqual`. The IDE should give warning about this line. 



##########
flink-ml-python/pyflink/ml/recommendation/tests/test_swing.py:
##########
@@ -0,0 +1,149 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.table import Table
+from typing import List
+from py4j.protocol import Py4JJavaError:

Review Comment:
   Let's run this test locally to ensure it can compile.
   
   ```suggestion
   from py4j.protocol import Py4JJavaError
   ```



##########
flink-ml-python/pyflink/ml/recommendation/tests/test_swing.py:
##########
@@ -0,0 +1,149 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.table import Table
+from typing import List
+from py4j.protocol import Py4JJavaError:
+
+from pyflink.ml.recommendation.swing import Swing
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+
+# Tests Swing. 
+class SwingTest(PyFlinkMLTestCase):
+    def setUp(self):
+        super(SwingTest, self).setUp()
+        self.train_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (0, 11),
+                (0, 12),
+                (1, 13),
+                (1, 12),
+                (2, 10),
+                (2, 11),
+                (2, 12),
+                (3, 13),
+                (3, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.wrong_type_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (1, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.INT(), Types.LONG()])
+            ))
+
+        self.none_value_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (None, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.expected_data = [
+            [10, '11,0.058845768947156235;12,0.058845768947156235'],
+            [11, '10,0.058845768947156235;12,0.058845768947156235'],
+            [12, '13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235'],
+            [13, '12,0.09134833828228624']]
+
+    def test_param(self):
+        swing = Swing()
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(100, swing.k)
+        self.assertEqual(10, swing.min_user_items)

Review Comment:
   There is no `min_user_items` and `max_user_items` defined in `Swing`. Please make sure this test is passed.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1107984853


##########
docs/content/docs/operators/recommendation/swing.md:
##########
@@ -0,0 +1,194 @@
+---
+title: "Swing"
+type: docs
+aliases:
+- /operators/recommendation/swing.html
+---
+
+<!--
+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.
+-->
+
+## Swing
+
+An AlgoOperator which implements the Swing algorithm.
+
+Swing is an item recall algorithm. 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 $i$, 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. 
+
+See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for Recommendation in
+E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.
+
+### Input Columns
+
+| Param name | Type | Default  | Description |
+|:-----------|:-----|:---------|:------------|
+| itemCol    | Long | `"item"` | Item id.    |
+| userCol    | Long | `"user"` | User id     |
+### Output Columns
+
+| Param name | Type   | Default        | Description                                                                 |
+|:-----------|:-------|:---------------|:----------------------------------------------------------------------------|
+| itemCol    | Long   | `"prediction"` | Item id.                                                                    |
+| outputCol  | String | `"output"`     | Recommendations and their score. (e.g. "item_1,0.9;item_2,0.7;item_3,0.35") |

Review Comment:
   What about using the following description?
   
   `Top k similar items and their corresponding scores (e.g. "item_1,0.9;item_2,0.7;item_3,0.35").`



##########
docs/content/docs/operators/recommendation/swing.md:
##########
@@ -0,0 +1,194 @@
+---
+title: "Swing"
+type: docs
+aliases:
+- /operators/recommendation/swing.html
+---
+
+<!--
+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.
+-->
+
+## Swing
+
+An AlgoOperator which implements the Swing algorithm.
+
+Swing is an item recall algorithm. 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 $i$, 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. 
+
+See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for Recommendation in
+E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.
+
+### Input Columns
+
+| Param name | Type | Default  | Description |
+|:-----------|:-----|:---------|:------------|
+| itemCol    | Long | `"item"` | Item id.    |
+| userCol    | Long | `"user"` | User id     |
+### Output Columns
+
+| Param name | Type   | Default        | Description                                                                 |
+|:-----------|:-------|:---------------|:----------------------------------------------------------------------------|
+| itemCol    | Long   | `"prediction"` | Item id.                                                                    |

Review Comment:
   The default value of `itemCol` should be `"item"`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.TestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 org.apache.commons.lang3.exception.ExceptionUtils;
+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.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table inputTable;
+
+    @Before
+    public void before() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        List<Row> inputRows =
+                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),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 10L),
+                                Row.of(4L, 11L),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 13L)));
+        inputTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                inputRows,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            BasicTypeInfo.LONG_TYPE_INFO,
+                                            BasicTypeInfo.LONG_TYPE_INFO
+                                        },
+                                        new String[] {"user_id", "item_id"})));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        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")));
+
+        results.sort(Comparator.comparing(o -> o.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testInputWithIllegalDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")

Review Comment:
   Let's use the default value when possible, e.g., `itemCol`, `userCol` and `outputCol`. Using the default value would simplify the test and also shows how users would use the presented algorithms.
   
   Same for other tests.



##########
flink-ml-python/pyflink/ml/recommendation/swing.py:
##########
@@ -0,0 +1,215 @@
+################################################################################
+#  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 pyflink.ml.common.param import HasOutputCol
+from pyflink.ml.param import Param, StringParam, IntParam, FloatParam, ParamValidators
+from pyflink.ml.recommendation.common import JavaRecommendationAlgoOperator
+from pyflink.ml.wrapper import JavaWithParams
+
+
+class _SwingParams(
+    JavaWithParams,
+    HasOutputCol
+):
+    """
+    Params for :class:`Swing`.
+    """
+
+    USER_COL: Param[str] = StringParam(
+        "user_col",
+        "User column name.",
+        "user",
+        ParamValidators.not_null())
+
+    ITEM_COL: Param[str] = StringParam(
+        "item_col",
+        "Item column name.",
+        "item",
+        ParamValidators.not_null())
+
+    K: Param[int] = IntParam(
+        "k",
+        "The max number of similar items to output for each item.",
+        100,
+        ParamValidators.gt(0))
+
+    MAX_USER_NUM_PER_ITEM: Param[int] = IntParam(
+        "max_user_num_per_item",
+        "The max number of users(purchasers) for each item. If the number of users "
+        + "is greater than this value, then only maxUserNumPerItem users will "
+        + "be sampled and used in the computation of similarity between two items.",
+        1000,
+        ParamValidators.gt(0))
+
+    MIN_USER_BEHAVIOR: Param[int] = IntParam(
+        "min_user_behavior",
+        "The min number of items that a user purchases. If the items purchased by a user is "
+        + "smaller than this value, then this user is filtered out and will not be used in the "
+        + "computation.",
+        10,
+        ParamValidators.gt(0))
+
+    MAX_USER_BEHAVIOR: Param[int] = IntParam(
+        "max_user_behavior",
+        "The max number of items for a user purchases. If the items purchased by a user is "
+        + "greater than this value, then this user is filtered out and will not be used in the "
+        + "computation.",
+        1000,
+        ParamValidators.gt(0))
+
+    ALPHA1: Param[int] = IntParam(
+        "alpha1",
+        "Smooth factor for number of users that have purchased one item. The higher alpha1 is,"
+        + " the less purchasing behavior contributes to the similarity score.",
+        15,
+        ParamValidators.gt_eq(0))
+
+    ALPHA2: Param[int] = IntParam(
+        "alpha2",
+        "Smooth factor for number of users that have purchased the two target items. The higher "
+        + "alpha2 is, the less purchasing behavior contributes to the similarity score.",
+        0,
+        ParamValidators.gt_eq(0))
+
+    BETA: Param[float] = FloatParam(
+        "beta",
+        "Decay factor for number of users that have purchased one item. The higher beta is, the "
+        + "less purchasing behavior contributes to the similarity score.",
+        0.3,
+        ParamValidators.gt_eq(0))
+
+    def __init__(self, java_params):
+        super(_SwingParams, self).__init__(java_params)
+
+    def set_user_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.USER_COL, value))
+
+    def get_user_col(self) -> str:
+        return self.get(self.USER_COL)
+
+    def set_item_col(self, value: str):
+        return typing.cast(_SwingParams, self.set(self.ITEM_COL, value))
+
+    def get_item_col(self) -> str:
+        return self.get(self.ITEM_COL)
+
+    def set_k(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.K, value))
+
+    def get_k(self) -> int:
+        return self.get(self.K)
+
+    def set_max_user_num_per_item(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_NUM_PER_ITEM, value))
+
+    def get_max_user_num_per_item(self) -> int:
+        return self.get(self.MAX_USER_NUM_PER_ITEM)
+
+    def set_min_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MIN_USER_BEHAVIOR, value))
+
+    def get_min_user_behavior(self) -> int:
+        return self.get(self.MIN_USER_BEHAVIOR)
+
+    def set_max_user_behavior(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.MAX_USER_BEHAVIOR, value))
+
+    def get_max_user_behavior(self) -> int:
+        return self.get(self.MAX_USER_BEHAVIOR)
+
+    def set_alpha1(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA1, value))
+
+    def get_alpha1(self) -> int:
+        return self.get(self.ALPHA1)
+
+    def set_alpha2(self, value: int):
+        return typing.cast(_SwingParams, self.set(self.ALPHA2, value))
+
+    def get_alpha2(self) -> int:
+        return self.get(self.ALPHA2)
+
+    def set_beta(self, value: float):
+        return typing.cast(_SwingParams, self.set(self.BETA, value))
+
+    def get_beta(self) -> float:
+        return self.get(self.BETA)
+
+    @property
+    def user_col(self) -> str:
+        return self.get_user_col()
+
+    @property
+    def item_col(self) -> str:
+        return self.get_item_col()
+
+    @property
+    def k(self) -> int:
+        return self.get_k()
+
+    @property
+    def max_user_num_per_item(self) -> int:
+        return self.get_max_user_num_per_item()
+
+    @property
+    def min_user_behavior(self) -> int:
+        return self.get_min_user_behavior()
+
+    @property
+    def max_user_behavior(self) -> int:
+        return self.get_max_user_behavior()
+
+    @property
+    def alpha1(self) -> int:
+        return self.get_alpha1()
+
+    @property
+    def alpha2(self) -> float:
+        return self.get_alpha2()
+
+    @property
+    def beta(self) -> float:
+        return self.get_beta()
+
+
+class Swing(JavaRecommendationAlgoOperator, _SwingParams):
+    """
+    An AlgoOperator which implements the Swing algorithm.
+
+    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 u
+    and user v have purchased the same commodity i , they will form a relationship
+    diagram similar to a swing. If u and v have purchased commodity j in
+    addition to i, it is supposed i and j are similar.
+
+    This implementation is based on the algorithm proposed in the paper: "Large Scale Product

Review Comment:
   Let's make the python doc consistent with java doc.
   
   Same for others.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.TestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 org.apache.commons.lang3.exception.ExceptionUtils;
+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.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table inputTable;
+
+    @Before
+    public void before() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        List<Row> inputRows =
+                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),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 10L),
+                                Row.of(4L, 11L),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 13L)));
+        inputTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                inputRows,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            BasicTypeInfo.LONG_TYPE_INFO,
+                                            BasicTypeInfo.LONG_TYPE_INFO
+                                        },
+                                        new String[] {"user_id", "item_id"})));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        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")));
+
+        results.sort(Comparator.comparing(o -> o.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testInputWithIllegalDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testInputWithNull() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();

Review Comment:
   Let's avoid using print in unit test since it cannot be checked.



##########
docs/content/docs/operators/recommendation/swing.md:
##########
@@ -0,0 +1,194 @@
+---
+title: "Swing"
+type: docs
+aliases:
+- /operators/recommendation/swing.html
+---
+
+<!--
+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.
+-->
+
+## Swing
+
+An AlgoOperator which implements the Swing algorithm.
+
+Swing is an item recall algorithm. 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 $i$, 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. 
+
+See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for Recommendation in
+E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.
+
+### Input Columns
+
+| Param name | Type | Default  | Description |
+|:-----------|:-----|:---------|:------------|
+| itemCol    | Long | `"item"` | Item id.    |
+| userCol    | Long | `"user"` | User id     |

Review Comment:
   nit: The description usually ends with a `.`.
   
   Same for `outputCol`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.TestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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 org.apache.commons.lang3.exception.ExceptionUtils;
+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.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/** Tests {@link Swing}. */
+public class SwingTest {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table inputTable;
+
+    @Before
+    public void before() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        List<Row> inputRows =
+                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),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 10L),
+                                Row.of(4L, 11L),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 13L)));
+        inputTable =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                inputRows,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                            BasicTypeInfo.LONG_TYPE_INFO,
+                                            BasicTypeInfo.LONG_TYPE_INFO
+                                        },
+                                        new String[] {"user_id", "item_id"})));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        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")));
+
+        results.sort(Comparator.comparing(o -> o.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            assertEquals(expect.getField(0), result.getField(0));
+            assertEquals(expect.getField(1), itemRankScore);
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(1000, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMaxUserNumPerItem(500)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(500, swing.getMaxUserNumPerItem());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test
+    public void testInputWithIllegalDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+
+        try {
+            Table[] swingResultTables =
+                    new Swing()
+                            .setItemCol("item_id")
+                            .setUserCol("user_id")
+                            .setOutputCol("item_score")
+                            .setMinUserBehavior(1)
+                            .transform(data);
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            assertEquals(IllegalArgumentException.class, e.getClass());
+            assertEquals("The types of user and item must be Long.", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testInputWithNull() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+
+        try {
+            swingResultTables[0].execute().print();
+            fail();
+        } catch (RuntimeException e) {
+            Throwable exception = ExceptionUtils.getRootCause(e);
+            assertEquals(RuntimeException.class, exception.getClass());
+            assertEquals("Data of user and item column must not be null.", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(inputTable);
+        Table output = swingResultTables[0];
+
+        assertEquals(
+                Arrays.asList("item_id", "item_score"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testTransform() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setMinUserBehavior(2)
+                        .setMaxUserBehavior(3);
+        Table[] swingResultTables = swing.transform(inputTable);
+        Table outputTable = swingResultTables[0];
+        List<Row> results = IteratorUtils.toList(outputTable.execute().collect());
+        compareResultAndExpected(results);
+    }
+
+    @Test
+    public void testSaveLoadAndTransform() throws Exception {
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Swing loadedSwing =
+                TestUtils.saveAndReload(tEnv, swing, tempFolder.newFolder().getAbsolutePath());
+        Table outputTable = loadedSwing.transform(inputTable)[0];
+        List<Row> results = IteratorUtils.toList(outputTable.execute().collect());
+        compareResultAndExpected(results);
+    }
+
+    @Test
+    public void getParams() {

Review Comment:
   The code snippet here seems useless. Could you delete it?



##########
docs/content/docs/operators/recommendation/swing.md:
##########
@@ -0,0 +1,194 @@
+---
+title: "Swing"
+type: docs
+aliases:
+- /operators/recommendation/swing.html
+---
+
+<!--
+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.
+-->
+
+## Swing
+
+An AlgoOperator which implements the Swing algorithm.
+
+Swing is an item recall algorithm. 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 $i$, 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. 
+
+See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for Recommendation in
+E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.
+
+### Input Columns
+
+| Param name | Type | Default  | Description |
+|:-----------|:-----|:---------|:------------|
+| itemCol    | Long | `"item"` | Item id.    |
+| userCol    | Long | `"user"` | User id     |
+### Output Columns
+
+| Param name | Type   | Default        | Description                                                                 |
+|:-----------|:-------|:---------------|:----------------------------------------------------------------------------|
+| itemCol    | Long   | `"prediction"` | Item id.                                                                    |
+| outputCol  | String | `"output"`     | Recommendations and their score. (e.g. "item_1,0.9;item_2,0.7;item_3,0.35") |
+
+### Parameters
+
+Below are the parameters required by `Swing`.
+
+| Key               | Default    | Type    | Required | Description                                                                                                                                                                                                                                                                                                                                                                                               |
+|:------------------|:-----------|:--------|:---------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| userCol           | `"user"`   | String  | no       | User column name.                                                                                                                                                                                                                                                                                                                                                                                         |
+| itemCol           | `"item"`   | String  | no       | Item column name.                                                                                                                                                                                                                                                                                                                                                                                         |
+| maxUserNumPerItem | `1000`     | Integer | no       | The max number of user(purchasers) for each item. If the number of user is larger than this value, then only maxUserNumPerItem users will be sampled and considered in the computation of similarity between two items.                                                                                                                                                                                   |
+| k                 | `100`      | Integer | no       | The max number of similar items to output for each item.                                                                                                                                                                                                                                                                                                                                                  |
+| minUserBehavior   | `10`       | Integer | no       | The min number of items for a user purchases. If the items purchased by a user is smaller than this value, then this user is filtered out while gathering data. This can affect the speed of the computation. Set minUserBehavior larger in case the swing recommendation progresses very slowly.                                                                                                         |
+| maxUserBehavior   | `1000`     | Integer | no       | The max number of items for a user purchases. If the items purchased by a user is larger than this value, then this user is filtered out while gathering data. This can affect the speed of the computation. Set maxUserBehavior smaller in case the swing recommendation progresses very slowly. The IllegalArgumentException is raised if the value of maxUserBehavior is smaller than minUserBehavior. |
+| alpha1            | `15`       | Integer | no       | Smooth factor for number of users that have purchased one item. The higher alpha1 is, the less purchasing behavior contributes to the similarity score.                                                                                                                                                                                                                                                   |
+| alpha2            | `0`        | Integer | no       | Smooth factor for number of users that have purchased the two target items. The higher alpha2 is, the less purchasing behavior contributes to the similarity score.                                                                                                                                                                                                                                       |
+| beta              | `0.3`      | Double  | no       | Decay factor for number of users that have purchased one item. The higher beta is, the less purchasing behavior contributes to the similarity score.                                                                                                                                                                                                                                                      |
+| outputCol         | `"output"` | String  | no       | Output column name.                                                                                                                                                                                                                                                                                                                                                                                       |
+
+### Examples
+
+{{< tabs examples >}}
+
+{{< tab "Java">}}
+
+```java
+package org.apache.flink.ml.examples.recommendation;
+
+import org.apache.flink.ml.recommendation.swing.Swing;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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.flink.util.CloseableIterator;
+
+/**
+ * Simple program that creates a Swing instance and uses it to generate recommendations for items.
+ */
+public class SwingExample {
+    public static void main(String[] args) {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        // Generates input data.
+        DataStream<Row> inputStream =
+                env.fromElements(
+                        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));
+
+        Table inputTable = tEnv.fromDataStream(inputStream).as("user", "item");
+
+        // Creates a Swing object and initializes its parameters.
+        Swing swing = new Swing().setUserCol("user").setItemCol("item").setMinUserBehavior(1);
+
+        // Transforms the data.
+        Table[] outputTable = swing.transform(inputTable);
+
+        // Extracts and displays the result of swing algorithm.
+        for (CloseableIterator<Row> it = outputTable[0].execute().collect(); it.hasNext(); ) {
+            Row row = it.next();
+
+            long mainItem = row.getFieldAs(0);
+            String itemRankScore = row.getFieldAs(1);
+
+            System.out.printf("mainItem %d, recommendedItem %s\n", mainItem, itemRankScore);

Review Comment:
   Let's update it as:
   
   `System.out.printf("item: %d, top-k similar items: %s\n", mainItem, itemRankScore);`
   
   Same for python/java examples.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
zhipeng93 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1105746948


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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

Review Comment:
   nit: there is a redundant space before `, they will...`



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity between items in Swing is defined as
+ *
+ * <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

Review Comment:
   Let's simplify the description here.
   
   `See "<a href="https://arxiv.org/pdf/2010.05525.pdf">Large Scale Product Graph Construction for Recommendation in E-commerce</a>" by Xiaoyong Yang, Yadong Zhu and Yi Zhang.`



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity between items in Swing is defined as
+ *
+ * <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. (<a
+ * href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String userCol = getUserCol();
+        final String itemCol = getItemCol();
+        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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    String.format("The maxUserBehavior must be larger or equal to minUserBehavior. "
+                                    + "The current value: maxUserBehavior=%d, minUserBehavior=%d.",
+                            getMaxUserBehavior(),
+                            getMinUserBehavior())
+            );
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> purchasingBehavior =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    Long userId = (Long) row.getFieldAs(userCol);
+                                    Long itemId = (Long) row.getFieldAs(itemCol);
+                                    if (userId == null || itemId == null) {
+                                        throw new RuntimeException(
+                                                "Data of user and item column must not be null.");

Review Comment:
   nit: How about updating the error message as `User or item column must not be null.`?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity between items in Swing is defined as
+ *
+ * <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. (<a
+ * href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String userCol = getUserCol();
+        final String itemCol = getItemCol();
+        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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    String.format("The maxUserBehavior must be larger or equal to minUserBehavior. "
+                                    + "The current value: maxUserBehavior=%d, minUserBehavior=%d.",
+                            getMaxUserBehavior(),
+                            getMinUserBehavior())
+            );
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> purchasingBehavior =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    Long userId = (Long) row.getFieldAs(userCol);
+                                    Long itemId = (Long) row.getFieldAs(itemCol);
+                                    if (userId == null || itemId == null) {
+                                        throw new RuntimeException(
+                                                "Data of user and item column must not be null.");
+                                    }
+                                    return Tuple2.of(userId, itemId);
+                                })
+                        .returns(Types.TUPLE(Types.LONG, Types.LONG));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Map<Long, String>>> userBehavior =
+                purchasingBehavior
+                        .keyBy(tuple -> tuple.f0)
+                        .transform(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                userBehavior
+                        .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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially, The appended column
+     * is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userAndPurchasedItems = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAndPurchasedItemsState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userAndPurchasedItems.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            items.forEach(
+                                    (item, nullValue) ->
+                                            output.collect(
+                                                    new StreamRecord<>(
+                                                            new Tuple3<>(user, item, items))));
+                        }
+                    });
+
+            userAndPurchasedItemsState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userAndPurchasedItems.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap<>();
+                userAndPurchasedItems.putIfAbsent(user, items);

Review Comment:
   Is there any case that `user` does exist in the map? If no, we probably should use `put()`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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 org.apache.commons.lang3.exception.ExceptionUtils;
+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;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests {@link Swing}.
+ */
+public class SwingTest {
+    @Rule
+    public final TemporaryFolder tempFolder = new TemporaryFolder();
+    private StreamExecutionEnvironment env;
+    private StreamTableEnvironment tEnv;
+    private Table trainData;
+
+    @Before
+    public void before() {
+        env = TestUtils.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+        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),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 10L),
+                                Row.of(4L, 11L),
+                                Row.of(4L, 12L),
+                                Row.of(4L, 13L)));
+        trainData =
+                tEnv.fromDataStream(
+                        env.fromCollection(
+                                trainRows,
+                                new RowTypeInfo(
+                                        new TypeInformation[] {
+                                                BasicTypeInfo.LONG_TYPE_INFO,
+                                                BasicTypeInfo.LONG_TYPE_INFO
+                                        },
+                                        new String[] {"user_id", "item_id"})));
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        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")));
+
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));

Review Comment:
   nit: This could be simplified as `results.sort(Comparator.comparingLong(o -> o.getFieldAs(0)));`



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity between items in Swing is defined as
+ *
+ * <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. (<a
+ * href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String userCol = getUserCol();
+        final String itemCol = getItemCol();
+        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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    String.format("The maxUserBehavior must be larger or equal to minUserBehavior. "
+                                    + "The current value: maxUserBehavior=%d, minUserBehavior=%d.",
+                            getMaxUserBehavior(),
+                            getMinUserBehavior())
+            );
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> purchasingBehavior =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    Long userId = (Long) row.getFieldAs(userCol);
+                                    Long itemId = (Long) row.getFieldAs(itemCol);
+                                    if (userId == null || itemId == null) {
+                                        throw new RuntimeException(
+                                                "Data of user and item column must not be null.");
+                                    }
+                                    return Tuple2.of(userId, itemId);
+                                })
+                        .returns(Types.TUPLE(Types.LONG, Types.LONG));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Map<Long, String>>> userBehavior =
+                purchasingBehavior
+                        .keyBy(tuple -> tuple.f0)
+                        .transform(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                userBehavior
+                        .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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially, The appended column
+     * is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userAndPurchasedItems = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAndPurchasedItemsState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userAndPurchasedItems.forEach(
+                    (user, items) -> {
+                        if (items.size() >= minUserItemInteraction
+                                && items.size() <= maxUserItemInteraction) {
+                            items.forEach(
+                                    (item, nullValue) ->
+                                            output.collect(
+                                                    new StreamRecord<>(
+                                                            new Tuple3<>(user, item, items))));
+                        }
+                    });
+
+            userAndPurchasedItemsState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userAndPurchasedItems.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap<>();
+                userAndPurchasedItems.putIfAbsent(user, items);
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAndPurchasedItemsState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAndPurchasedItemsState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAndPurchasedItemsState, "userAndPurchasedItemsState")
+                    .ifPresent(
+                            stat -> {
+                                userAndPurchasedItems = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAndPurchasedItemsState.update(Collections.singletonList(userAndPurchasedItems));
+        }
+    }
+
+    /**
+     * Calculates similarity between items and keep top k similar items of each target item.
+     */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+            BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userAndPurchasedItems = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemAndPurchasers = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userAndPurchasedItemsState;
+        private ListState<Map<Long, Map<Long, String>>> itemAndPurchasersState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+
+        /**
+         * Alpha1 and alpha2 are integers larger or equal to zero. Though they by definition should be greater than

Review Comment:
   Do you think we should explain this in the java doc of the `Swing` class? 
   
   Because reader may get interested in the behavior of `Swing` when `alpha2` is zero, without reading the implementation here.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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",
+                    "User column name. The type of user column must be Long.",
+                    "user",
+                    ParamValidators.notNull());
+
+    Param<String> ITEM_COL =
+            new StringParam(
+                    "itemCol",
+                    "Item column name. The type of item column must be Long.",
+                    "item",
+                    ParamValidators.notNull());
+
+    Param<Integer> MAX_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "The max number of user(purchasers) for each item. If the number of user "
+                            + "is larger than this value, then only maxUserNumPerItem users will "
+                            + "be sampled and considered in the computation of similarity between two items.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item. If an item has "

Review Comment:
   `If an item has...` seems an explanation of `The max number of similar items to output for each item.`. 
   Do you think it provides more information?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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",
+                    "User column name. The type of user column must be Long.",
+                    "user",
+                    ParamValidators.notNull());
+
+    Param<String> ITEM_COL =
+            new StringParam(
+                    "itemCol",
+                    "Item column name. The type of item column must be Long.",
+                    "item",
+                    ParamValidators.notNull());
+
+    Param<Integer> MAX_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "The max number of user(purchasers) for each item. If the number of user "
+                            + "is larger than this value, then only maxUserNumPerItem users will "
+                            + "be sampled and considered in the computation of similarity between two items.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item. If an item has "
+                            + "more than k recommendations, the first k similar items will be kept",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of items for a user purchases. If the items purchased by a user is smaller than "

Review Comment:
   It is not a `fit` process. Can you replace the `fit` here?
   
   `This can affect the speed of the computation. The best value depends on the nature of the problem.` This sentence does not introduce extra accurate information --- It is hard to tell how would this parameter affect performance and accuracy, and thus cannot guide users. Moreover, it is unclear what is `nature` of the probem.
   
   So I would just suggest remove it here. I am also fine if you can make it more accurate.
   
   Same for MAX_USER_BEHAVIOR.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity between items in Swing is defined as
+ *
+ * <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. (<a
+ * href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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) {
+        Preconditions.checkArgument(inputs.length == 1);
+        final String userCol = getUserCol();
+        final String itemCol = getItemCol();
+        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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    String.format("The maxUserBehavior must be larger or equal to minUserBehavior. "
+                                    + "The current value: maxUserBehavior=%d, minUserBehavior=%d.",
+                            getMaxUserBehavior(),
+                            getMinUserBehavior())
+            );
+        }
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+
+        SingleOutputStreamOperator<Tuple2<Long, Long>> purchasingBehavior =
+                tEnv.toDataStream(inputs[0])
+                        .map(
+                                row -> {
+                                    Long userId = (Long) row.getFieldAs(userCol);
+                                    Long itemId = (Long) row.getFieldAs(itemCol);
+                                    if (userId == null || itemId == null) {
+                                        throw new RuntimeException(
+                                                "Data of user and item column must not be null.");
+                                    }
+                                    return Tuple2.of(userId, itemId);
+                                })
+                        .returns(Types.TUPLE(Types.LONG, Types.LONG));
+
+        SingleOutputStreamOperator<Tuple3<Long, Long, Map<Long, String>>> userBehavior =
+                purchasingBehavior
+                        .keyBy(tuple -> tuple.f0)
+                        .transform(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        new String[] {getItemCol(), getOutputCol()});
+
+        DataStream<Row> output =
+                userBehavior
+                        .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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially, The appended column
+     * is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.

Review Comment:
   nit: `userItemsMap` and `itemUsersMap` needs to be updated. It seems inconsistent with existing code.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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",
+                    "User column name. The type of user column must be Long.",

Review Comment:
   We usually do not explain the type of a column when describing the param. Let's follow the existing convention.
   
   Same for ITEM_COL.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] weibozhao commented on pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
weibozhao commented on PR #192:
URL: https://github.com/apache/flink-ml/pull/192#issuecomment-1373235362

   Thanks for the update.  I Left some comments below.


-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] zhipeng93 commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "zhipeng93 (via GitHub)" <gi...@apache.org>.
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1093937923


##########
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:
   Thanks for your advice, `CalculateSimilarity` sounds like a name of method and `ComputingSimilarItems` is much better.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] jiangxin369 commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "jiangxin369 (via GitHub)" <gi...@apache.org>.
jiangxin369 commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1094587044


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((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 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, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Set<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = new ArrayList<>(entry.getValue());
+                Long user = entry.getKey();
+                if (items.size() < minUserItemInteraction
+                        || items.size() > maxUserItemInteraction) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Set<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                Set<Long> value = new LinkedHashSet<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItemInteraction) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                for (long user : x.keySet()) {
+                                    List<Long> itemList = x.get(user);
+                                    userItemsMap.put(user, new LinkedHashSet<>(itemList));
+                                }
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map<Long, List<Long>> userItemsList = new HashMap<>(userItemsMap.size());
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                userItemsList.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+            }
+            userAllItemsMapState.update(Collections.singletonList(userItemsList));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, List<Long>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userLocalItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);

Review Comment:
   You can construct the HashSet from ArrayList by `new HashSet<>(items)` to simplify the code. So as the other HashSet construction.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        DataStream<Row> dataStream = env.fromCollection(trainRows, trainDataTypeInfo);
+        trainData = tEnv.fromDataStream(dataStream);
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore, expect.getField(1));
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+        Table[] swingResultTables =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1)
+                        .transform(data);
+
+        swingResultTables[0].execute().collect();
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void testNumberFormat() {
+        List<Row> rows =
+                new ArrayList<>(
+                        Arrays.asList(
+                                Row.of(0L, 10L),
+                                Row.of(null, 12L),
+                                Row.of(1L, 13L),
+                                Row.of(3L, 12L)));
+        DataStream<Row> dataStream = env.fromCollection(rows, trainDataTypeInfo);
+        Table data = tEnv.fromDataStream(dataStream);
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(data);
+        swingResultTables[0].execute().print();
+    }
+
+    @Test
+    public void testOutputSchema() {
+        Swing swing =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+
+        assertEquals(
+                Arrays.asList("item_id", "item_score"),
+                output.getResolvedSchema().getColumnNames());
+    }
+
+    @Test
+    public void testFewerItemCase() {
+        Swing swing = new Swing().setItemCol("item_id").setUserCol("user_id").setMinUserBehavior(5);
+        Table[] swingResultTables = swing.transform(trainData);
+        Table output = swingResultTables[0];
+        List<Row> results = IteratorUtils.toList(output.execute().collect());
+        Assert.assertEquals(0, results.size());
+    }
+
+    @Test
+    public void testTransform() throws Exception {

Review Comment:
   The `throws Exception` can be removed.



##########
flink-ml-python/pyflink/ml/lib/recommendation/tests/test_swing.py:
##########
@@ -0,0 +1,165 @@
+################################################################################
+#  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 pyflink.common import Types
+from pyflink.table import Table
+from typing import List
+from py4j.protocol import Py4JJavaError:
+
+from pyflink.ml.lib.recommendation.swing import Swing
+from pyflink.ml.tests.test_utils import PyFlinkMLTestCase
+
+
+# Tests Swing. 
+class SwingTest(PyFlinkMLTestCase):
+    def setUp(self):
+        super(SwingTest, self).setUp()
+        self.train_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (0, 11),
+                (0, 12),
+                (1, 13),
+                (1, 12),
+                (2, 10),
+                (2, 11),
+                (2, 12),
+                (3, 13),
+                (3, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.wrong_type_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (1, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.INT(), Types.LONG()])
+            ))
+
+        self.none_value_data = self.t_env.from_data_stream(
+            self.env.from_collection([
+                (0, 10),
+                (None, 11),
+                (2, 12)
+            ],
+                type_info=Types.ROW_NAMED(
+                    ['user', 'item'],
+                    [Types.LONG(), Types.LONG()])
+            ))
+
+        self.expected_data = [
+            [10, '11,0.058845768947156235;12,0.058845768947156235'],
+            [11, '10,0.058845768947156235;12,0.058845768947156235'],
+            [12, '13,0.09134833828228624;10,0.058845768947156235;11,0.058845768947156235'],
+            [13, '12,0.09134833828228624']]
+
+    def test_param(self):
+        swing = Swing()
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(100, swing.k)
+        self.assertEqual(10, swing.min_user_items)
+        self.assertEqual(1000, swing.max_user_items)
+        self.assertEqual(15, swing.alpha1)
+        self.assertEqual(0, swing.alpha2)
+        self.assertEqual(0.3, swing.beta, delta=1e-9)
+
+        swing.set_item_col("item_1") \
+            .set_user_col("user_1") \
+            .set_k(20) \
+            .set_min_user_behavior(20) \
+            .set_max_user_behavior(50) \
+            .set_alpha1(5) \
+            .set_alpha2(1) \
+            .set_beta(0.35)
+
+        self.assertEqual("item", swing.item_col)
+        self.assertEqual("user", swing.user_col)
+        self.assertEqual(20, swing.k)
+        self.assertEqual(20, swing.min_user_items)
+        self.assertEqual(50, swing.max_user_items)
+        self.assertEqual(5, swing.alpha1)
+        self.assertEqual(1, swing.alpha2)
+        self.assertEqual(0.35, swing.beta, delta=1e-9)
+
+    def test_output_schema(self):
+        swing = Swing() \
+            .set_item_col('test_item') \
+            .set_user_col('test_user') \
+            .set_output_col("item_score")
+
+        output = swing.transform(self.train_data.alias(['test_user', 'test_item']))[0]
+        self.assertEqual(
+            ['test_item', 'item_score'],
+            output.get_schema().get_field_names())
+
+    def test_transform(self):
+        swing = Swing().set_min_user_behavior(1)
+        output = swing.transform(self.train_data)[0]
+        self.verify_output_result(
+            output,
+            swing.get_item_col(),
+            output.get_schema().get_field_names(),
+            self.expected_data)
+
+    def test_save_load_and_transform(self):
+        swing = Swing().set_min_user_behavior(1)
+        reloaded_swing = self.save_and_reload(swing)
+        output = reloaded_swing.transform(self.train_data)[0]
+        self.verify_output_result(
+            output,
+            swing.get_item_col(),
+            output.get_schema().get_field_names(),
+            self.expected_data)
+
+    def test_data_type(self):
+        try:
+            swing = Swing().set_min_user_behavior(1)
+            output = swing.transform(self.wrong_type_data)[0]
+            self.t_env.to_data_stream(output).execute_and_collect()
+        except Py4JJavaError:
+            pass
+
+    def test_number_format(self):

Review Comment:
   It seems that`test_data_type` and `test_number_format` test nothing because you are not checking the error message. Btw, I'm not sure whether we need these tests in Python, because it is already covered in Java tests.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((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 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, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Set<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = new ArrayList<>(entry.getValue());
+                Long user = entry.getKey();
+                if (items.size() < minUserItemInteraction
+                        || items.size() > maxUserItemInteraction) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Set<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                Set<Long> value = new LinkedHashSet<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItemInteraction) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                for (long user : x.keySet()) {
+                                    List<Long> itemList = x.get(user);
+                                    userItemsMap.put(user, new LinkedHashSet<>(itemList));
+                                }
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map<Long, List<Long>> userItemsList = new HashMap<>(userItemsMap.size());
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                userItemsList.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+            }
+            userAllItemsMapState.update(Collections.singletonList(userItemsList));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, List<Long>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userLocalItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction = (HashSet<Long>) userItemsMap.get(u).clone();
+                        interaction.retainAll(userItemsMap.get(v));
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Set<Long> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers);
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();
+
+            for (long u : allUsers) {
+                double guess = rand.nextDouble();
+                if (guess < prob) {
+                    userList.add(u);
+                    sampleSize--;
+                }
+                totalSize--;
+                prob = (double) sampleSize / totalSize;
+            }
+
+            return userList;
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple3<Long, Long, List<Long>>> streamRecord)
+                throws Exception {
+            Tuple3<Long, Long, List<Long>> tuple3 = streamRecord.getValue();
+            long user = tuple3.f0;
+            long item = tuple3.f1;
+            List<Long> items = tuple3.f2;
+
+            if (!userItemsMap.containsKey(user)) {
+                HashSet<Long> itemSet = new HashSet<>(items.size());
+                itemSet.addAll(items);
+                userItemsMap.put(user, itemSet);
+            }
+
+            if (!itemUsersMap.containsKey(item)) {
+                itemUsersMap.put(item, new HashSet<>());
+            }
+            itemUsersMap.get(item).add(user);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userLocalItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userLocalItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userLocalItemsMapState, "userLocalItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            userItemsMap.put(k, itemSet);
+                                        });
+                            });
+
+            itemUsersMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "itemUsersMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+                    .ifPresent(
+                            x -> {
+                                itemUsersMap = new HashMap<>(x.size());
+                                x.forEach(
+                                        (k, v) -> {
+                                            HashSet<Long> itemSet = new HashSet<>(v.size());
+                                            itemSet.addAll(v);
+                                            itemUsersMap.put(k, itemSet);
+                                        });
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map tmpUerItemsMap = new HashMap(userItemsMap.size());

Review Comment:
   Let's replace the `Map` with `Map<Long, List<Long>>` to avoid raw use. So as the other parameterized class like `ArrayList`, `Map`. 



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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(4);
+        env.enableCheckpointing(100);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        tEnv = StreamTableEnvironment.create(env);
+        DataStream<Row> dataStream = env.fromCollection(trainRows, trainDataTypeInfo);
+        trainData = tEnv.fromDataStream(dataStream);
+    }
+
+    private void compareResultAndExpected(List<Row> results) {
+        results.sort((o1, o2) -> Long.compare(o1.getFieldAs(0), o2.getFieldAs(0)));
+
+        for (int i = 0; i < results.size(); i++) {
+            Row result = results.get(i);
+            String itemRankScore = result.getFieldAs(1);
+            Row expect = expectedScoreRows.get(i);
+            Assert.assertEquals(result.getField(0), expect.getField(0));
+            Assert.assertEquals(itemRankScore, expect.getField(1));
+        }
+    }
+
+    @Test
+    public void testParam() {
+        Swing swing = new Swing();
+
+        assertEquals("item", swing.getItemCol());
+        assertEquals("user", swing.getUserCol());
+        assertEquals(100, swing.getK());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(1000, swing.getMaxUserBehavior());
+        assertEquals(15, swing.getAlpha1());
+        assertEquals(0, swing.getAlpha2());
+        assertEquals(0.3, swing.getBeta(), 1e-9);
+
+        swing.setItemCol("item_1")
+                .setUserCol("user_1")
+                .setK(20)
+                .setMinUserBehavior(10)
+                .setMaxUserBehavior(50)
+                .setAlpha1(5)
+                .setAlpha2(1)
+                .setBeta(0.35);
+
+        assertEquals("item_1", swing.getItemCol());
+        assertEquals("user_1", swing.getUserCol());
+        assertEquals(20, swing.getK());
+        assertEquals(10, swing.getMinUserBehavior());
+        assertEquals(50, swing.getMaxUserBehavior());
+        assertEquals(5, swing.getAlpha1());
+        assertEquals(1, swing.getAlpha2());
+        assertEquals(0.35, swing.getBeta(), 1e-9);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testDataType() {
+        List<Row> rows =
+                new ArrayList<>(Arrays.asList(Row.of(0, "10"), Row.of(1, "11"), Row.of(2, "")));
+        DataStream<Row> dataStream =
+                env.fromCollection(
+                        rows,
+                        new RowTypeInfo(
+                                new TypeInformation[] {
+                                    BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO
+                                },
+                                new String[] {"user_id", "item_id"}));
+        Table data = tEnv.fromDataStream(dataStream);
+        Table[] swingResultTables =
+                new Swing()
+                        .setItemCol("item_id")
+                        .setUserCol("user_id")
+                        .setOutputCol("item_score")
+                        .setMinUserBehavior(1)
+                        .transform(data);
+
+        swingResultTables[0].execute().collect();
+    }
+
+    @Test(expected = RuntimeException.class)

Review Comment:
   Would it be better if we use `try { ...; fail();} catch { assert errorMsg}` instead of using `expected = RuntimeException.class`? It avoids the situation that unexpected RuntimeException happens but the test is passed. You can refer to `testFitOnEmptyData` in `VarianceThresholdSelectorTest`.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.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;
+    RowTypeInfo trainDataTypeInfo =
+            new RowTypeInfo(
+                    new TypeInformation[] {
+                        BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO
+                    },
+                    new String[] {"user_id", "item_id"});
+    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();

Review Comment:
   Please append the `.disableGenericTypes()` to the execution config to ensure that no inefficient serialization is used. We will add this config for all algorithms in the future.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());

Review Comment:
   Would it be simpler to replace the `(Number) row.getFieldAs(itemCol)).longValue()` with `(Long) row.getFieldAs(userCol)`?



##########
flink-ml-python/pyflink/examples/ml/recommendation/swing_example.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 pyflink.common import Types
+from pyflink.datastream import StreamExecutionEnvironment
+from pyflink.table import StreamTableEnvironment, Schema
+from pyflink.table.types import DataTypes
+
+from pyflink.ml.lib.recommendation.swing import Swing
+
+# Creates a new StreamExecutionEnvironment.
+env = StreamExecutionEnvironment.get_execution_environment()
+
+# Creates a StreamTableEnvironment.
+t_env = StreamTableEnvironment.create(env)
+
+# Generates input data.
+input_table = t_env.from_data_stream(
+    env.from_collection([
+        (0, 10),
+        (0, 11),
+        (0, 12),
+        (1, 13),
+        (1, 12),
+        (2, 10),
+        (2, 11),
+        (2, 12),
+        (3, 13),
+        (3, 12)
+    ],
+        type_info=Types.ROW_NAMED(
+        ['user', 'item'],
+        [Types.LONG(), Types.LONG()])
+    ))
+
+# Creates a swing object and initialize its parameters.
+swing = Swing()\
+    .set_item_col('item')\
+    .set_user_col("user")\
+    .set_min_user_behavior(1)
+
+# Transforms the data to Swing algorithm result.
+swingTable = swing.transform(input_table)

Review Comment:
   How about following the existing naming style and naming it `output`? And almost all the other python examples contain a docstring like `Simple program that creates a ...`, could you add it?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/SwingParams.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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_USER_NUM_PER_ITEM =
+            new IntParam(
+                    "maxUserNumPerItem",
+                    "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.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> K =
+            new IntParam(
+                    "k",
+                    "The max number of similar items to output for each item.",
+                    100,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MIN_USER_BEHAVIOR =
+            new IntParam(
+                    "minUserBehavior",
+                    "The min number of interaction behavior between item and user.",
+                    10,
+                    ParamValidators.gt(0));
+
+    Param<Integer> MAX_USER_BEHAVIOR =
+            new IntParam(
+                    "maxUserBehavior",
+                    "The max number of interaction behavior between item and user. "
+                            + "The algorithm filters out activate users.",
+                    1000,
+                    ParamValidators.gt(0));
+
+    Param<Integer> ALPHA1 =
+            new IntParam(
+                    "alpha1",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher alpha1 is, the smaller weight each user gets.",
+                    15,
+                    ParamValidators.gtEq(0));
+
+    Param<Integer> ALPHA2 =
+            new IntParam(
+                    "alpha2",
+                    "This parameter is used to calculate similarity of users. "
+                            + "The higher alpha2 is, the less the similarity score is.",
+                    0,
+                    ParamValidators.gtEq(0));
+
+    Param<Double> BETA =
+            new DoubleParam(
+                    "beta",
+                    "This parameter is used to calculate weight of each user. "
+                            + "The higher beta is, the weight is subject to exponential decay.",
+                    0.3,
+                    ParamValidators.gtEq(0));
+
+    default String getUserCol() {
+        return get(USER_COL);
+    }
+
+    default T setUserCol(String value) {
+        return set(USER_COL, value);
+    }
+
+    default String getItemCol() {
+        return get(ITEM_COL);
+    }
+
+    default T setItemCol(String value) {
+        return set(ITEM_COL, value);
+    }
+
+    default int getK() {
+        return get(K);
+    }
+
+    default T setK(Integer value) {
+        return set(K, value);
+    }
+
+    default int getMaxUserNumPerItem() {
+        return get(MAX_USER_NUM_PER_ITEM);
+    }
+
+    default T setMaxUserNumPerItem(Integer value) {

Review Comment:
   Please add tests for `setMaxUserNumPerItem` in `test_param()` and `testParam()`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * 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(
+                                            ((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 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, List<Long>>>
+            implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple3<Long, Long, List<Long>>>,
+                    BoundedOneInput {
+        private final int minUserItemInteraction;
+        private final int maxUserItemInteraction;
+
+        private Map<Long, Set<Long>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, List<Long>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                List<Long> items = new ArrayList<>(entry.getValue());
+                Long user = entry.getKey();
+                if (items.size() < minUserItemInteraction
+                        || items.size() > maxUserItemInteraction) {
+                    continue;
+                }
+                for (Long item : items) {
+                    output.collect(new StreamRecord<>(new Tuple3<>(user, item, items)));
+                }
+            }
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Set<Long> items = userItemsMap.get(user);
+
+            if (items == null) {
+                Set<Long> value = new LinkedHashSet<>();
+                value.add(item);
+                userItemsMap.put(user, value);
+            } else {
+                if (items.size() <= maxUserItemInteraction) {
+                    items.add(item);
+                }
+            }
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(Types.LONG, Types.LIST(Types.LONG))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            x -> {
+                                userItemsMap = new HashMap<>(x.size());
+                                for (long user : x.keySet()) {
+                                    List<Long> itemList = x.get(user);
+                                    userItemsMap.put(user, new LinkedHashSet<>(itemList));
+                                }
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            Map<Long, List<Long>> userItemsList = new HashMap<>(userItemsMap.size());
+            for (Entry<Long, Set<Long>> entry : userItemsMap.entrySet()) {
+                userItemsList.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+            }
+            userAllItemsMapState.update(Collections.singletonList(userItemsList));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, List<Long>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, HashSet<Long>> userItemsMap = new HashMap<>();
+        private Map<Long, HashSet<Long>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, List<Long>>> userLocalItemsMapState;
+        private ListState<Map<Long, List<Long>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';

Review Comment:
   Let's declare these two delimiters as `final`, or just remove the declaration because they are only used in one place.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "Fanoid (via GitHub)" <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1100890942


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

Review Comment:
   nits: Since we have just used `Types.LONG` and `Types.STRING`, let's continue using them here to make it consistent.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1101092374


##########
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(
+                                            ((Number) row.getFieldAs(userCol)).longValue(),
+                                            ((Number) row.getFieldAs(itemCol)).longValue());
+                                })
+                        .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<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.entrySet().stream()
+                    .forEach(
+                            entry -> {
+                                Long user = entry.getKey();
+                                Map<Long, String> items = entry.getValue();
+                                if (items.size() >= minUserItemInteraction
+                                        && items.size() <= maxUserItemInteraction) {
+                                    items.entrySet().stream()
+                                            .forEach(
+                                                    item ->
+                                                            output.collect(
+                                                                    new StreamRecord<>(
+                                                                            new Tuple3<>(
+                                                                                    user,
+                                                                                    item.getKey(),
+                                                                                    items))));
+                                }
+                            });
+
+            userAllItemsMapState.clear();
+        }
+
+        @Override
+        public void processElement(StreamRecord<Tuple2<Long, Long>> element) {
+            Tuple2<Long, Long> userAndItem = element.getValue();
+            long user = userAndItem.f0;
+            long item = userAndItem.f1;
+            Map<Long, String> items = userItemsMap.get(user);
+
+            if (items == null) {
+                items = new LinkedHashMap();
+            }
+
+            if (items.size() <= maxUserItemInteraction) {
+                items.put(item, null);
+            }
+
+            userItemsMap.put(user, items);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws Exception {
+            super.initializeState(context);
+            userAllItemsMapState =
+                    context.getOperatorStateStore()
+                            .getListState(
+                                    new ListStateDescriptor<>(
+                                            "userAllItemsMapState",
+                                            Types.MAP(
+                                                    Types.LONG,
+                                                    Types.MAP(Types.LONG, Types.STRING))));
+
+            OperatorStateUtils.getUniqueElement(userAllItemsMapState, "userAllItemsMapState")
+                    .ifPresent(
+                            stat -> {
+                                userItemsMap = stat;
+                            });
+        }
+
+        @Override
+        public void snapshotState(StateSnapshotContext context) throws Exception {
+            super.snapshotState(context);
+            userAllItemsMapState.update(Collections.singletonList(userItemsMap));
+        }
+    }
+
+    /** Calculates top N similar items of each item. */
+    private static class ComputingSimilarItems extends AbstractStreamOperator<Row>
+            implements OneInputStreamOperator<Tuple3<Long, Long, Map<Long, String>>, Row>,
+                    BoundedOneInput {
+
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+        private Map<Long, Map<Long, String>> itemUsersMap = new HashMap<>();
+        private ListState<Map<Long, Map<Long, String>>> userLocalItemsMapState;
+        private ListState<Map<Long, Map<Long, String>>> itemUsersMapState;
+
+        private final int k;
+        private final int maxUserNumPerItem;
+        private final int alpha1;
+        private final int alpha2;
+        private final double beta;
+
+        private static Character commaDelimiter = ',';
+        private static Character semicolonDelimiter = ';';
+
+        private ComputingSimilarItems(
+                int k, int maxUserNumPerItem, int alpha1, int alpha2, double beta) {
+            this.k = k;
+            this.maxUserNumPerItem = maxUserNumPerItem;
+            this.alpha1 = alpha1;
+            this.alpha2 = alpha2;
+            this.beta = beta;
+        }
+
+        @Override
+        public void endInput() throws Exception {
+
+            Map<Long, Double> userWeights = new HashMap<>(userItemsMap.size());
+            userItemsMap.forEach(
+                    (k, v) -> {
+                        int count = v.size();
+                        userWeights.put(k, calculateWeight(count));
+                    });
+
+            for (long mainItem : itemUsersMap.keySet()) {
+                List<Long> userList = sampleUserList(itemUsersMap.get(mainItem), maxUserNumPerItem);
+                HashMap<Long, Double> id2swing = new HashMap<>();
+
+                for (int i = 0; i < userList.size(); i++) {
+                    long u = userList.get(i);
+                    for (int j = i + 1; j < userList.size(); j++) {
+                        long v = userList.get(j);
+                        HashSet<Long> interaction =
+                                (HashSet<Long>) new HashSet<>(userItemsMap.get(u).keySet()).clone();
+                        interaction.retainAll(userItemsMap.get(v).keySet());
+                        if (interaction.size() == 0) {
+                            continue;
+                        }
+                        double similarity =
+                                (userWeights.get(u)
+                                        * userWeights.get(v)
+                                        / (alpha2 + interaction.size()));
+                        for (long simItem : interaction) {
+                            if (simItem == mainItem) {
+                                continue;
+                            }
+                            double itemSimilarity =
+                                    id2swing.getOrDefault(simItem, 0.0) + similarity;
+                            id2swing.putIfAbsent(simItem, itemSimilarity);
+                        }
+                    }
+                }
+
+                ArrayList<Tuple2<Long, Double>> itemAndScore = new ArrayList<>();
+                id2swing.forEach((key, value) -> itemAndScore.add(Tuple2.of(key, value)));
+
+                itemAndScore.sort((o1, o2) -> Double.compare(o2.f1, o1.f1));
+
+                if (itemAndScore.size() == 0) {
+                    continue;
+                }
+
+                int itemNums = Math.min(k, itemAndScore.size());
+                StringBuilder sbd = new StringBuilder();
+                for (int i = 0; i < itemNums; i++) {
+                    sbd.append(itemAndScore.get(i).f0).append(commaDelimiter);
+                    sbd.append(itemAndScore.get(i).f1).append(semicolonDelimiter);
+                }
+                String itemList = sbd.substring(0, sbd.length() - 1);
+
+                output.collect(new StreamRecord<>(Row.of(mainItem, itemList)));
+            }
+
+            userLocalItemsMapState.clear();
+            itemUsersMapState.clear();
+        }
+
+        private double calculateWeight(int size) {
+            return (1.0 / Math.pow(alpha1 + size, beta));
+        }
+
+        private static List<Long> sampleUserList(Map<Long, String> allUsers, int sampleSize) {
+            int totalSize = allUsers.size();
+            if (totalSize < sampleSize) {
+                return new ArrayList(allUsers.keySet());
+            }
+
+            List<Long> userList = new ArrayList<>(totalSize);
+            double prob = (double) sampleSize / totalSize;
+            Random rand = new Random();

Review Comment:
   No problem.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] Fanoid commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
Fanoid commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1052819663


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {

Review Comment:
   `userAlpha` and `userBeta` are only used in this method. How about put them just here?  And then we can make this method `static`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						userItemsMap.put(k, itemSet);
+					});
+				});
+
+			itemUsersMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"itemUsersMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+				.ifPresent(x -> {
+					itemUsersMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						itemUsersMap.put(k, itemSet);
+					});
+				});
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				tmpUerItemsMap.put(k, new ArrayList <>(v));
+			});
+			userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));

Review Comment:
   Just curious if using `Collections.singletonList(userItemsMap)` works here?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());

Review Comment:
   Maybe use `new HashSet<>(items)` which is simple.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+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.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,
+					SWING_ITEM_RANK,
+					SWING_ITEM_SCORE));
+		DataStream <Row> output =
+			BroadcastUtils.withBroadcastStream(
+				Collections.singletonList(data),
+				Collections.singletonMap(broadcastModelKey, model),
+				inputList -> {
+					DataStream input = inputList.get(0);
+					return input.flatMap(
+						new PredictLabelFunction(broadcastModelKey, predictItemCol),
+						outputTypeInfo
+					);
+				});
+		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);
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		ReadWriteUtils.saveModelData(
+			model,
+			path,
+			new SwingModelData.ModelDataEncoder()
+			//new Encoder <Row>() {

Review Comment:
   Unused codes should be removed.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.linalg.DenseVector;
+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.CollectionUtils;
+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 java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class SwingTest {
+	@Rule
+	public final TemporaryFolder tempFolder = new TemporaryFolder();
+	private StreamExecutionEnvironment env;
+	private StreamTableEnvironment tEnv;
+	private Table trainData;
+	private Table testData;
+	private static final List <Row> trainRows =
+		new ArrayList <>(
+			Arrays.asList(
+				Row.of("a", "apple"),
+				Row.of("a", "orange"),
+				Row.of("a", "juice"),
+				Row.of("b", "milk"),
+				Row.of("b", "juice"),
+				Row.of("c", "apple"),
+				Row.of("c", "orange"),
+				Row.of("c", "juice"),
+				Row.of("d", "milk"),
+				Row.of("d", "juice")
+			)
+		);
+
+	private static final List <Row> testRows =
+		new ArrayList <>(
+			Arrays.asList(
+				Row.of("milk"),
+				Row.of("apple"),
+				Row.of("orange"),
+				Row.of("juice")
+			)
+		);
+
+	private static final List <Row> expectedScoreRows =
+		new ArrayList <>(
+			Arrays.asList(
+				Row.of("milk", "juice", 0, 1.3015095),
+				Row.of("apple", "orange", 0, 1.0717735),
+				Row.of("apple", "juice", 0, 1.0717735),
+				Row.of("orange", "apple", 0, 1.0717735),
+				Row.of("orange", "juice", 0, 1.0717735),
+				Row.of("juice", " milk", 0, 1.3015095),
+				Row.of("juice", "orange", 0, 1.0717735),
+				Row.of("juice", "apple", 0, 1.0717735)
+			)
+		);
+
+	@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.STRING())
+				.column("f1", DataTypes.STRING())
+				.build();
+		DataStream <Row> dataStream = env.fromCollection(trainRows);
+		trainData = tEnv.fromDataStream(dataStream, schema).as("user", "item");
+		
+		Schema testSchema =
+			Schema.newBuilder()
+				.column("f0", DataTypes.STRING())
+				.build();
+		DataStream <Row> testStream = env.fromCollection(testRows);
+		testData = tEnv.fromDataStream(testStream, testSchema).as("item");
+	}
+
+	@Test
+	public void testParam() {
+		Swing swing = new Swing();
+
+		assertEquals("item", swing.getItemCol());
+		assertEquals("user", swing.getUserCol());
+		assertEquals(10, swing.getTopN());
+		assertEquals(10, swing.getMinUserItems());
+		assertEquals(1000, swing.getMaxUserItems());
+
+		swing.setItemCol("item_1")
+			.setUserCol("user_1")
+			.setTopN(20)
+			.setMinUserItems(10)
+			.setMaxUserItems(50);
+
+		assertEquals("item_1", swing.getItemCol());
+		assertEquals("user_1", swing.getUserCol());
+		assertEquals(20, swing.getTopN());
+		assertEquals(10, swing.getMinUserItems());
+		assertEquals(50, swing.getMaxUserItems());
+
+	}
+
+	@Test
+	public void testOutputSchema() {
+		Table input = trainData;
+		Swing swing = new Swing()
+			.setItemCol("item")
+			.setUserCol("user")
+			.setMinUserItems(1);
+		SwingModel swingModel = swing.fit(trainData);
+		Table output = swingModel.transform(input)[0];
+
+		assertEquals(
+			Arrays.asList("user","item", "SWING_ITEM_NAME", "SWING_ITEM_RANK", "SWING_ITEM_SCORE"),
+			output.getResolvedSchema().getColumnNames());
+	}
+
+	@Test
+	public void testFewerItemCase() {
+		Swing swing = new Swing()
+			.setItemCol("item")
+			.setUserCol("user")
+			.setMinUserItems(5);
+		SwingModel swingModel = swing.fit(trainData);
+		Table output = swingModel.setItemCol("item").transform(testData)[0];
+		List <Row> results = IteratorUtils.toList(output.execute().collect());
+		Assert.assertEquals(0, results.size());
+	}
+
+	@Test
+	public void testFitAndPredict() throws Exception {
+		Swing swing = new Swing()
+			.setItemCol("item")
+			.setUserCol("user")
+			.setMinUserItems(1);
+		SwingModel swingModel = swing.fit(trainData);
+		Table output = swingModel.setItemCol("item").transform(testData)[0];
+		List <Row> results = IteratorUtils.toList(output.execute().collect());
+		for(int i=0;i<results.size();i++){

Review Comment:
   Maybe a better way to compare two `List<Row>` is to sort two lists with some deterministic comparator, then compare element pairs one by one.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();

Review Comment:
   The overall format seems not consistent with existing codes. Could you please do the format check as in [1]?
   
   [1] https://github.com/apache/flink-ml/tree/master/.github/workflows



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+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.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,

Review Comment:
   Is it expected that these column names are fixed?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();

Review Comment:
   The `similarity` and `interaction` between `u` and `v` seems irrelevant with `mainItem`.  Then, for different `mainItem`s, the `similarity` and `interaction` between two same users are computed multiple times.  
   
   Is it possible to precompute them or memorize them for repeated computation?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),

Review Comment:
   Seems unnecessary to use `new ArrayList <String>`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+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.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,
+					SWING_ITEM_RANK,
+					SWING_ITEM_SCORE));
+		DataStream <Row> output =
+			BroadcastUtils.withBroadcastStream(
+				Collections.singletonList(data),
+				Collections.singletonMap(broadcastModelKey, model),
+				inputList -> {
+					DataStream input = inputList.get(0);
+					return input.flatMap(
+						new PredictLabelFunction(broadcastModelKey, predictItemCol),
+						outputTypeInfo
+					);
+				});
+		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);
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		ReadWriteUtils.saveModelData(
+			model,
+			path,
+			new SwingModelData.ModelDataEncoder()
+			//new Encoder <Row>() {
+			//	StringSerializer stringSerializer = new StringSerializer();
+			//	ListSerializer <String> stringListSerializer = new ListSerializer <>(new StringSerializer());
+			//	ListSerializer <Float> floatListSerializer = new ListSerializer <>(new FloatSerializer());
+			//
+			//	@Override
+			//	public void encode(Row row, OutputStream outputStream) throws IOException {
+			//		DataOutputView dataOutputView = new DataOutputViewStreamWrapper(outputStream);
+			//
+			//		stringSerializer.serialize((String) row.getField(0), dataOutputView);
+			//		stringListSerializer.serialize((List <String>) row.getField(1), dataOutputView);
+			//		floatListSerializer.serialize((List <Float>) row.getField(2), dataOutputView);
+			//	}
+			//}
+		);
+	}
+
+	public static SwingModel load(StreamTableEnvironment tEnv, String path) throws IOException {
+		SwingModel model = ReadWriteUtils.loadStageParam(path);
+		Table modelDataTable =
+			ReadWriteUtils.loadModelData(
+				tEnv,
+				path,
+				new SwingModelData.ModelDataDecoder()
+				//new SimpleStreamFormat <Row>() {

Review Comment:
   Unused codes should be removed.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>

Review Comment:
   Can you add some links for this algorithm?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());

Review Comment:
   I'm not quite sure, but does it work if just use `new HashMap<>(x)`?  Is deep clone necessary here?



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;

Review Comment:
   Maybe you need `putIfAbsent`.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModelData.java:
##########
@@ -0,0 +1,106 @@
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.clustering.kmeans.KMeansModelData;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class SwingModelData {
+	public String mainItem;
+	public ArrayList <String> items;

Review Comment:
   Just a little curious, are there any specific reasons to use implementation classes(`ArrayList`) instead of the interface (`List`)?  There are also `HashSet` instead of `Set` in other places.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.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.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+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.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.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator 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.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@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);
+	}
+
+	/**
+	 * Append 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 append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();

Review Comment:
   Copy constructor of `HashSet` is usually preferred than `clone`.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add Estimator and Transformer for Swing

Posted by GitBox <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1053893147


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+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.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,

Review Comment:
   No, It's better to be a parameter set by user. Should I use `HasOutputCol` and combine a sequence of recommendation items, rank and score?



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-ml] vacaly commented on a diff in pull request #192: [FLINK-30451] Add AlgoOperator for Swing

Posted by "vacaly (via GitHub)" <gi...@apache.org>.
vacaly commented on code in PR #192:
URL: https://github.com/apache/flink-ml/pull/192#discussion_r1105551447


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/swing/Swing.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.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.stream.Collectors;
+
+/**
+ * An AlgoOperator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall algorithm. 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 similarity
+ * between items in Swing is defined as
+ *
+ * <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.
+ * (<a href="https://arxiv.org/pdf/2010.05525.pdf">https://arxiv.org/pdf/2010.05525.pdf</a>)
+ */
+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.");
+        }
+
+        if (getMaxUserBehavior() < getMinUserBehavior()) {
+            throw new IllegalArgumentException(
+                    "The maxUserBehavior must be larger or equal to minUserBehavior.");
+        }
+
+        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(
+                                "collectingUserBehavior",
+                                Types.TUPLE(
+                                        Types.LONG,
+                                        Types.LONG,
+                                        Types.MAP(Types.LONG, Types.STRING)),
+                                new CollectingUserBehavior(
+                                        getMinUserBehavior(), getMaxUserBehavior()));
+
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        new TypeInformation[] {Types.LONG, Types.STRING},
+                        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);
+    }
+
+    /**
+     * Collects user behavior data and appends to the input table.
+     *
+     * <p>During the process, this operator collects users and all items he/she has purchased, and
+     * its input table must be bounded. Because Flink doesn't support type info of `Set` officially,
+     * The appended column is `Map` contains items as key and maps null value.
+     */
+    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;
+
+        // Maps a user id to a set of items. Because ListState cannot keep values of type `Set`,
+        // we use `Map<Long, String>` with null values instead. So does `userItemsMap` and
+        // `itemUsersMap` in `ComputingSimilarItems`.
+        private Map<Long, Map<Long, String>> userItemsMap = new HashMap<>();
+
+        private ListState<Map<Long, Map<Long, String>>> userAllItemsMapState;
+
+        private CollectingUserBehavior(int minUserItemInteraction, int maxUserItemInteraction) {
+            this.minUserItemInteraction = minUserItemInteraction;
+            this.maxUserItemInteraction = maxUserItemInteraction;
+        }
+
+        @Override
+        public void endInput() {
+
+            userItemsMap.forEach(

Review Comment:
   Yes, but flink type doesn't support `Set`. 



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org