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

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

Reply via email to