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


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

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



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to