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

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

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


##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						userItemsMap.put(k, itemSet);
+					});
+				});
+
+			itemUsersMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"itemUsersMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(itemUsersMapState, "itemUsersMapState")
+				.ifPresent(x -> {
+					itemUsersMap = new HashMap <>(x.size());
+					x.forEach((k, v) -> {
+						HashSet <String> itemSet = new HashSet <>(v.size());
+						itemSet.addAll(v);
+						itemUsersMap.put(k, itemSet);
+					});
+				});
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			Map tmpUerItemsMap = new HashMap(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				tmpUerItemsMap.put(k, new ArrayList <>(v));
+			});
+			userItemsMapState.update(Collections.singletonList(tmpUerItemsMap));

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,
+					SWING_ITEM_RANK,
+					SWING_ITEM_SCORE));
+		DataStream <Row> output =
+			BroadcastUtils.withBroadcastStream(
+				Collections.singletonList(data),
+				Collections.singletonMap(broadcastModelKey, model),
+				inputList -> {
+					DataStream input = inputList.get(0);
+					return input.flatMap(
+						new PredictLabelFunction(broadcastModelKey, predictItemCol),
+						outputTypeInfo
+					);
+				});
+		return new Table[] {tEnv.fromDataStream(output)};
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		ReadWriteUtils.saveModelData(
+			model,
+			path,
+			new SwingModelData.ModelDataEncoder()
+			//new Encoder <Row>() {

Review Comment:
   Unused codes should be removed.



##########
flink-ml-lib/src/test/java/org/apache/flink/ml/recommendation/SwingTest.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.util.TestUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.IteratorUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class SwingTest {
+	@Rule
+	public final TemporaryFolder tempFolder = new TemporaryFolder();
+	private StreamExecutionEnvironment env;
+	private StreamTableEnvironment tEnv;
+	private Table trainData;
+	private Table testData;
+	private static final List <Row> trainRows =
+		new ArrayList <>(
+			Arrays.asList(
+				Row.of("a", "apple"),
+				Row.of("a", "orange"),
+				Row.of("a", "juice"),
+				Row.of("b", "milk"),
+				Row.of("b", "juice"),
+				Row.of("c", "apple"),
+				Row.of("c", "orange"),
+				Row.of("c", "juice"),
+				Row.of("d", "milk"),
+				Row.of("d", "juice")
+			)
+		);
+
+	private static final List <Row> testRows =
+		new ArrayList <>(
+			Arrays.asList(
+				Row.of("milk"),
+				Row.of("apple"),
+				Row.of("orange"),
+				Row.of("juice")
+			)
+		);
+
+	private static final List <Row> expectedScoreRows =
+		new ArrayList <>(
+			Arrays.asList(
+				Row.of("milk", "juice", 0, 1.3015095),
+				Row.of("apple", "orange", 0, 1.0717735),
+				Row.of("apple", "juice", 0, 1.0717735),
+				Row.of("orange", "apple", 0, 1.0717735),
+				Row.of("orange", "juice", 0, 1.0717735),
+				Row.of("juice", " milk", 0, 1.3015095),
+				Row.of("juice", "orange", 0, 1.0717735),
+				Row.of("juice", "apple", 0, 1.0717735)
+			)
+		);
+
+	@Before
+	public void before() {
+		Configuration config = new Configuration();
+		config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true);
+		env = StreamExecutionEnvironment.getExecutionEnvironment(config);
+		env.getConfig().enableObjectReuse();
+		env.setParallelism(2);
+		env.enableCheckpointing(100);
+		env.setRestartStrategy(RestartStrategies.noRestart());
+		tEnv = StreamTableEnvironment.create(env);
+		Schema schema =
+			Schema.newBuilder()
+				.column("f0", DataTypes.STRING())
+				.column("f1", DataTypes.STRING())
+				.build();
+		DataStream <Row> dataStream = env.fromCollection(trainRows);
+		trainData = tEnv.fromDataStream(dataStream, schema).as("user", "item");
+		
+		Schema testSchema =
+			Schema.newBuilder()
+				.column("f0", DataTypes.STRING())
+				.build();
+		DataStream <Row> testStream = env.fromCollection(testRows);
+		testData = tEnv.fromDataStream(testStream, testSchema).as("item");
+	}
+
+	@Test
+	public void testParam() {
+		Swing swing = new Swing();
+
+		assertEquals("item", swing.getItemCol());
+		assertEquals("user", swing.getUserCol());
+		assertEquals(10, swing.getTopN());
+		assertEquals(10, swing.getMinUserItems());
+		assertEquals(1000, swing.getMaxUserItems());
+
+		swing.setItemCol("item_1")
+			.setUserCol("user_1")
+			.setTopN(20)
+			.setMinUserItems(10)
+			.setMaxUserItems(50);
+
+		assertEquals("item_1", swing.getItemCol());
+		assertEquals("user_1", swing.getUserCol());
+		assertEquals(20, swing.getTopN());
+		assertEquals(10, swing.getMinUserItems());
+		assertEquals(50, swing.getMaxUserItems());
+
+	}
+
+	@Test
+	public void testOutputSchema() {
+		Table input = trainData;
+		Swing swing = new Swing()
+			.setItemCol("item")
+			.setUserCol("user")
+			.setMinUserItems(1);
+		SwingModel swingModel = swing.fit(trainData);
+		Table output = swingModel.transform(input)[0];
+
+		assertEquals(
+			Arrays.asList("user","item", "SWING_ITEM_NAME", "SWING_ITEM_RANK", "SWING_ITEM_SCORE"),
+			output.getResolvedSchema().getColumnNames());
+	}
+
+	@Test
+	public void testFewerItemCase() {
+		Swing swing = new Swing()
+			.setItemCol("item")
+			.setUserCol("user")
+			.setMinUserItems(5);
+		SwingModel swingModel = swing.fit(trainData);
+		Table output = swingModel.setItemCol("item").transform(testData)[0];
+		List <Row> results = IteratorUtils.toList(output.execute().collect());
+		Assert.assertEquals(0, results.size());
+	}
+
+	@Test
+	public void testFitAndPredict() throws Exception {
+		Swing swing = new Swing()
+			.setItemCol("item")
+			.setUserCol("user")
+			.setMinUserItems(1);
+		SwingModel swingModel = swing.fit(trainData);
+		Table output = swingModel.setItemCol("item").transform(testData)[0];
+		List <Row> results = IteratorUtils.toList(output.execute().collect());
+		for(int i=0;i<results.size();i++){

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/SwingModel.java:
##########
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.serialization.Encoder;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.reader.SimpleStreamFormat;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.recommendation.SwingModelData.ModelDataDecoder;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Model which recommends item for top-N similar items using the model data computed by {@link Swing} .
+ */
+public class SwingModel implements Model <SwingModel>, SwingModelParams <SwingModel> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+	private Table modelDataTable;
+	private final String SWING_ITEM_NAME = "SWING_ITEM_NAME";
+	private final String SWING_ITEM_RANK = "SWING_ITEM_RANK";
+	private final String SWING_ITEM_SCORE = "SWING_ITEM_SCORE";
+
+	public SwingModel() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel setModelData(Table... inputs) {
+		modelDataTable = inputs[0];
+		return this;
+	}
+
+	@Override
+	public Table[] getModelData() {
+		return new Table[] {modelDataTable};
+	}
+
+	@Override
+	public Table[] transform(Table... inputs) {
+		Preconditions.checkArgument(inputs.length == 1);
+		final String predictItemCol = getItemCol();
+		final String broadcastModelKey = "broadcastModelKey";
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment();
+		DataStream <Row> data = tEnv.toDataStream(inputs[0]);
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+		RowTypeInfo outputTypeInfo =
+			new RowTypeInfo(
+				ArrayUtils.addAll(
+					inputTypeInfo.getFieldTypes(),
+					BasicTypeInfo.STRING_TYPE_INFO,
+					BasicTypeInfo.INT_TYPE_INFO,
+					BasicTypeInfo.FLOAT_TYPE_INFO),
+				ArrayUtils.addAll(inputTypeInfo.getFieldNames(),
+					SWING_ITEM_NAME,
+					SWING_ITEM_RANK,
+					SWING_ITEM_SCORE));
+		DataStream <Row> output =
+			BroadcastUtils.withBroadcastStream(
+				Collections.singletonList(data),
+				Collections.singletonMap(broadcastModelKey, model),
+				inputList -> {
+					DataStream input = inputList.get(0);
+					return input.flatMap(
+						new PredictLabelFunction(broadcastModelKey, predictItemCol),
+						outputTypeInfo
+					);
+				});
+		return new Table[] {tEnv.fromDataStream(output)};
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment();
+		DataStream <SwingModelData> model =SwingModelData.getDataStream(tEnv,modelDataTable);
+
+		ReadWriteUtils.saveModelData(
+			model,
+			path,
+			new SwingModelData.ModelDataEncoder()
+			//new Encoder <Row>() {
+			//	StringSerializer stringSerializer = new StringSerializer();
+			//	ListSerializer <String> stringListSerializer = new ListSerializer <>(new StringSerializer());
+			//	ListSerializer <Float> floatListSerializer = new ListSerializer <>(new FloatSerializer());
+			//
+			//	@Override
+			//	public void encode(Row row, OutputStream outputStream) throws IOException {
+			//		DataOutputView dataOutputView = new DataOutputViewStreamWrapper(outputStream);
+			//
+			//		stringSerializer.serialize((String) row.getField(0), dataOutputView);
+			//		stringListSerializer.serialize((List <String>) row.getField(1), dataOutputView);
+			//		floatListSerializer.serialize((List <Float>) row.getField(2), dataOutputView);
+			//	}
+			//}
+		);
+	}
+
+	public static SwingModel load(StreamTableEnvironment tEnv, String path) throws IOException {
+		SwingModel model = ReadWriteUtils.loadStageParam(path);
+		Table modelDataTable =
+			ReadWriteUtils.loadModelData(
+				tEnv,
+				path,
+				new SwingModelData.ModelDataDecoder()
+				//new SimpleStreamFormat <Row>() {

Review Comment:
   Unused codes should be removed.



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;
+							id2swing.put(simItem, itemSimilarity);
+						}
+					}
+				}
+
+				ArrayList <Tuple2 <String, Float>> itemAndScore = new ArrayList <>();
+				id2swing.forEach(
+					(key, value) -> itemAndScore.add(Tuple2.of(key, value))
+				);
+
+				itemAndScore.sort(new Comparator <Tuple2 <String, Float>>() {
+					@Override
+					public int compare(Tuple2 <String, Float> o1, Tuple2 <String, Float> o2) {
+						return 0 - Float.compare(o1.f1, o2.f1);
+					}
+				});
+
+				if (itemAndScore.size() == 0) {
+					continue;
+				}
+
+				int itemNums = topN > itemAndScore.size() ? itemAndScore.size() : topN;
+				String[] itemIds = new String[itemNums];
+				Float[] itemScores = new Float[itemNums];
+				for (int i = 0; i < itemNums; i++) {
+					itemIds[i] = itemAndScore.get(i).f0;
+					itemScores[i] = itemAndScore.get(i).f1;
+				}
+
+				output.collect(
+					new StreamRecord <>(
+						Row.of(
+							mainItem, new ArrayList <String>(Arrays.asList(itemIds)),
+							new ArrayList <Float>(Arrays.asList(itemScores))
+						)
+					)
+				);
+			}
+			userItemsMapState.clear();
+			itemUsersMapState.clear();
+		}
+
+		private float calculateWeight(int size) {
+			return (float) (1.0 / Math.pow(userAlpha + size, userBeta));
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple3 <String, String, List <String>>> streamRecord)
+			throws Exception {
+			Tuple3 <String, String, List <String>> tuple3 = streamRecord.getValue();
+			String user = tuple3.f0;
+			String item = tuple3.f1;
+			List <String> items = tuple3.f2;
+
+			if (!userItemsMap.containsKey(user)) {
+				HashSet <String> itemSet = new HashSet <>(items.size());
+				itemSet.addAll(items);
+				userItemsMap.put(user, itemSet);
+			}
+
+			if (!itemUsersMap.containsKey(item)) {
+				itemUsersMap.put(item, new HashSet <>());
+			}
+			itemUsersMap.get(item).add(user);
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> {
+					userItemsMap = new HashMap <>(x.size());

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();
+						interaction.retainAll(userItemsMap.get(v));
+						if (interaction.size() == 0) {
+							continue;
+						}
+						float similarity =
+							(float) (userWeights.get(u) * userWeights.get(v) / (alpha + interaction.size()));
+						for (String simItem : interaction) {
+							if (simItem.equals(mainItem)) {
+								continue;
+							}
+							float itemSimilarity = id2swing.getOrDefault(simItem, (float) 0) + similarity;

Review Comment:
   Maybe you need `putIfAbsent`.



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

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



##########
flink-ml-lib/src/main/java/org/apache/flink/ml/recommendation/Swing.java:
##########
@@ -0,0 +1,407 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.ml.recommendation;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.iteration.operator.OperatorStateUtils;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.internal.TableImpl;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * An Estimator which implements the Swing algorithm.
+ *
+ * <p>Swing is an item recall model. The topology of user-item graph usually can be described as user-item-user or
+ * item-user-item, which are like 'swing'. For example, if both user <em>u</em> and user <em>v</em> have purchased the
+ * same commodity <em>i</em> , they will form a relationship diagram similar to a swing. If <em>u</em> and <em>v</em>
+ * have purchased commodity <em>j</em> in addition to <em>i</em>, it is supposed <em>i</em> and <em>j</em> are
+ * similar.</p>
+ */
+public class Swing implements Estimator <Swing, SwingModel>, SwingParams <Swing> {
+	private final Map <Param <?>, Object> paramMap = new HashMap <>();
+
+	public Swing() {
+		ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+	}
+
+	@Override
+	public SwingModel fit(Table... inputs) {
+
+		final String userCol = getUserCol();
+		final String itemCol = getItemCol();
+		final int minUserItems = getMinUserItems();
+		final int maxUserItems = getMaxUserItems();
+		final ResolvedSchema schema = ((TableImpl) inputs[0]).getResolvedSchema();
+		final LogicalType userColType = schema.getColumn(userCol).get().getDataType().getLogicalType();
+		final LogicalType itemColType = schema.getColumn(itemCol).get().getDataType().getLogicalType();
+
+		if (!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(userColType).toLogicalType()) ||
+			!TypeCheckUtils.isCharacterString(InternalTypeInfo.of(itemColType).toLogicalType())) {
+			throw new IllegalArgumentException("Type of user and item column must be string.");
+		}
+
+		StreamTableEnvironment tEnv =
+			(StreamTableEnvironment) ((TableImpl) inputs[0])
+				.getTableEnvironment();
+
+		SingleOutputStreamOperator <Tuple2 <String, String>> itemUsers =
+			tEnv.toDataStream(inputs[0])
+				.map(row -> Tuple2.of((String) row.getFieldAs(userCol), (String) row.getFieldAs(itemCol)))
+				.returns(Types.TUPLE(Types.STRING, Types.STRING));
+
+		SingleOutputStreamOperator <Tuple3 <String, String, List <String>>> userAllItemsStream = itemUsers
+			.keyBy(tuple -> tuple.f0)
+			.transform("fillUserItemsTable",
+				Types.TUPLE(Types.STRING, Types.STRING, Types.LIST(Types.STRING)),
+				new BuildSwingData(minUserItems, maxUserItems));
+
+		SingleOutputStreamOperator <SwingModelData> similarity = userAllItemsStream
+			.keyBy(tuple -> tuple.f1)
+			.transform("calculateSimilarity",
+				Types.ROW(Types.STRING, Types.LIST(Types.STRING), Types.LIST(Types.FLOAT)),
+				new CalculateSimilarity(getTopN()))
+			.map(new MapFunction <Row, SwingModelData>() {
+				@Override
+				public SwingModelData map(Row value) throws Exception {
+					return new SwingModelData(value.getFieldAs(0), value.getFieldAs(1), value.getFieldAs(2));
+				}
+			});
+
+		SwingModel model = new SwingModel().setModelData(tEnv.fromDataStream(similarity));
+		ReadWriteUtils.updateExistingParams(model, getParamMap());
+		return model;
+	}
+
+	@Override
+	public Map <Param <?>, Object> getParamMap() {
+		return paramMap;
+	}
+
+	@Override
+	public void save(String path) throws IOException {
+		ReadWriteUtils.saveMetadata(this, path);
+	}
+
+	public static Swing load(StreamTableEnvironment tEnv, String path) throws IOException {
+		return ReadWriteUtils.loadStageParam(path);
+	}
+
+	/**
+	 * Append one column that records all items the user has purchased to the input table.
+	 *
+	 * <p>During the process, this operator collect users and all items a user has purchased into a map of list.
+	 * When the input is finished, this operator append the certain user-purchased-items list to each row. </p>
+	 */
+	private static class BuildSwingData
+		extends AbstractStreamOperator <Tuple3 <String, String, List <String>>>
+		implements OneInputStreamOperator <
+		Tuple2 <String, String>,
+		Tuple3 <String, String, List <String>>>,
+		BoundedOneInput {
+		final int minUserItems;
+		final int maxUserItems;
+
+		private Map <String, List <String>> userItemsMap = new HashMap <>();
+
+		private ListState <Map <String, List <String>>> userItemsMapState;
+
+		private BuildSwingData(int minUserItems, int maxUserItems) {
+			this.minUserItems = minUserItems;
+			this.maxUserItems = maxUserItems;
+		}
+
+		@Override
+		public void endInput() {
+
+			for (Entry <String, List <String>> entry :
+				userItemsMap.entrySet()) {
+				List <String> items = entry.getValue();
+				String user = entry.getKey();
+				if (items.size() < minUserItems || items.size() > maxUserItems) {
+					continue;
+				}
+				for (String item : items) {
+					output.collect(
+						new StreamRecord <>(
+							new Tuple3 <>(
+								user, item, items)));
+				}
+			}
+
+			userItemsMapState.clear();
+		}
+
+		@Override
+		public void processElement(StreamRecord <Tuple2 <String, String>> element) {
+			Tuple2 <String, String> userAndItem =
+				element.getValue();
+			String user = userAndItem.f0;
+			String item = userAndItem.f1;
+			List <String> items = userItemsMap.get(user);
+
+			if (items == null) {
+				ArrayList <String> value = new ArrayList <>();
+				value.add(item);
+				userItemsMap.put(user, value);
+			} else {
+				if (!items.contains(item)) {
+					items.add(item);
+				}
+			}
+		}
+
+		@Override
+		public void initializeState(StateInitializationContext context) throws Exception {
+			super.initializeState(context);
+			userItemsMapState =
+				context.getOperatorStateStore()
+					.getListState(
+						new ListStateDescriptor <>(
+							"userItemsMapState",
+							Types.MAP(
+								Types.STRING,
+								Types.LIST(
+									Types.STRING)))
+					);
+
+			OperatorStateUtils.getUniqueElement(userItemsMapState, "userItemsMapState")
+				.ifPresent(x -> userItemsMap = x);
+
+		}
+
+		@Override
+		public void snapshotState(StateSnapshotContext context) throws Exception {
+			super.snapshotState(context);
+			userItemsMapState.update(Collections.singletonList(userItemsMap));
+		}
+	}
+
+	/**
+	 * Calculate top N similar items of each item.
+	 */
+	private static class CalculateSimilarity
+		extends AbstractStreamOperator <Row>
+		implements OneInputStreamOperator <
+		Tuple3 <String, String, List <String>>,
+		Row>,
+		BoundedOneInput {
+
+		private Map <String, HashSet <String>> userItemsMap = new HashMap <>();
+		private Map <String, HashSet <String>> itemUsersMap = new HashMap <>();
+		private ListState <Map <String, List <String>>> userItemsMapState;
+		private ListState <Map <String, List <String>>> itemUsersMapState;
+
+		final private double alpha = 1.0;
+		final private double userAlpha = 5.0;
+		final private double userBeta = -0.35;
+		final private int topN;
+
+		private CalculateSimilarity(int topN) {this.topN = topN;}
+
+		@Override
+		public void endInput() throws Exception {
+
+			Map <String, Float> userWeights = new HashMap <>(userItemsMap.size());
+			userItemsMap.forEach((k, v) -> {
+				int count = v.size();
+				userWeights.put(k, calculateWeight(count));
+			});
+
+			for (String mainItem : itemUsersMap.keySet()) {
+				List <String> userList = new ArrayList(itemUsersMap.get(mainItem));
+				HashMap <String, Float> id2swing = new HashMap <>();
+
+				for (int i = 0; i < userList.size(); i++) {
+					String u = userList.get(i);
+					for (int j = i + 1; j < userList.size(); j++) {
+						String v = userList.get(j);
+						HashSet <String> interaction = (HashSet <String>) userItemsMap.get(u).clone();

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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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