Skip to content
This repository has been archived by the owner on Jul 30, 2020. It is now read-only.

Commit

Permalink
Added EditHistory recommendations
Browse files Browse the repository at this point in the history
  • Loading branch information
mschwarzer committed Jun 13, 2017
1 parent cd83743 commit db2a379
Show file tree
Hide file tree
Showing 11 changed files with 100,373 additions and 32 deletions.
138 changes: 138 additions & 0 deletions src/main/java/org/wikipedia/citolytics/edits/EditEvaluation.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
package org.wikipedia.citolytics.edits;

import org.apache.flink.api.common.functions.GroupReduceFunction;
import org.apache.flink.api.common.functions.JoinFunction;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.operators.DataSource;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.shaded.com.google.common.collect.Ordering;
import org.apache.flink.util.Collector;
import org.wikipedia.citolytics.WikiSimAbstractJob;
import org.wikipedia.citolytics.cpa.io.WikiDocumentDelimitedInputFormat;
import org.wikipedia.citolytics.cpa.types.RecommendationSet;
import org.wikipedia.citolytics.edits.operators.EditInputMapper;
import org.wikipedia.citolytics.edits.types.ArticleAuthorPair;
import org.wikipedia.citolytics.edits.types.AuthorArticlesList;
import org.wikipedia.citolytics.edits.types.CoEditList;
import org.wikipedia.citolytics.seealso.types.WikiSimComparableResult;
import org.wikipedia.citolytics.seealso.types.WikiSimComparableResultList;

import java.util.*;

/**
* Offline evaluation of article recommendations based on article edits by Wikipedia users. Authors contribute
* most likely to article that are related to each other. Therefore, we can utilize edits as gold standard for
* evaluating recommendations.
*
* Approach: Extract article-author pairs, find co-authored articles, order by count.
* Result will have article-recommendation-count triples.
*
* DataSource: stub-meta-history.xml-dump (available for each language)
*
* Problems:
* - Edits by bots (how to exclude them?, comment contains "bot")
*
*/
public class EditEvaluation extends WikiSimAbstractJob<RecommendationSet> {
String inputFilename;

public static void main(String[] args) throws Exception {
new EditEvaluation().start(args);
}

@Override
public void plan() throws Exception {
ParameterTool params = ParameterTool.fromArgs(args);

inputFilename = params.getRequired("input");
outputFilename = params.getRequired("output");

// Read Wikipedia Edit History XML Dump
DataSource<String> historyDump = env.readFile(new WikiDocumentDelimitedInputFormat(), inputFilename);


DataSet<ArticleAuthorPair> articleAuthorPairs = historyDump.flatMap(new EditInputMapper());

DataSet<AuthorArticlesList> authorArticlesList = articleAuthorPairs
.groupBy(1) // author id
.reduceGroup(new GroupReduceFunction<ArticleAuthorPair, AuthorArticlesList>() {
@Override
public void reduce(Iterable<ArticleAuthorPair> in, Collector<AuthorArticlesList> out) throws Exception {
Iterator<ArticleAuthorPair> iterator = in.iterator();
HashSet<String> articles = new HashSet<>();
ArticleAuthorPair pair = null;

while(iterator.hasNext()) {
pair = iterator.next();
articles.add(pair.getArticle());
}

out.collect(new AuthorArticlesList(pair.getAuthor(), new ArrayList<>(articles)));
}
});

DataSet<CoEditList> coEdits = articleAuthorPairs.join(authorArticlesList)
.where(1) // author id
.equalTo(0) // author id
.with(new JoinFunction<ArticleAuthorPair, AuthorArticlesList, CoEditList>() {
@Override
public CoEditList join(ArticleAuthorPair articleAuthorPair, AuthorArticlesList authorArticlesList) throws Exception {
return new CoEditList(articleAuthorPair.getArticle(), authorArticlesList.getArticleList());
}
});

DataSet<RecommendationSet> editRecommendations = coEdits
.groupBy(0) // article
.reduceGroup(new GroupReduceFunction<CoEditList, RecommendationSet>() {
@Override
public void reduce(Iterable<CoEditList> edits, Collector<RecommendationSet> out) throws Exception {
Iterator<CoEditList> iterator = edits.iterator();
CoEditList editList = null;
String articleName = null;
Map<String, Integer> coEditedArticles = new HashMap<>();
List<WikiSimComparableResult<Double>> recommendations = new ArrayList<>();

// Build recommendation set (count co-edits per article)
while(iterator.hasNext()) {
editList = iterator.next();

if(articleName == null)
articleName = editList.getArticle();

for(String article: editList.getList()) {
if(article.equals(articleName)) // Do not recommend the article itself
continue;

if(coEditedArticles.containsKey(article)) {
coEditedArticles.put(article, coEditedArticles.get(article) + 1);
} else {
coEditedArticles.put(article, 1);
}
}
}

for(String article: coEditedArticles.keySet()) {
double score = coEditedArticles.get(article);
int articleId = 0; // ignore ids
recommendations.add(new WikiSimComparableResult<>(article, score, articleId));
}

// Order recommendations
int topK = 10;
List<WikiSimComparableResult<Double>> orderedRecommendations = Ordering.natural().greatestOf(recommendations, topK);

out.collect(new RecommendationSet(
editList.getArticle(),
0, // ignore ids
new WikiSimComparableResultList(orderedRecommendations)
));
}
});

// coEdits.print();
// editRecommendations.print();

result = editRecommendations;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
package org.wikipedia.citolytics.edits.io;

import org.apache.flink.api.common.io.DelimitedInputFormat;
import org.apache.flink.configuration.Configuration;

import java.io.IOException;

/**
* DataSource format for Wikipedia stub-meta-history.xml dumps
*/
public class WikiStubMetaHistoryFormat extends DelimitedInputFormat<String> {

private static final long serialVersionUID = 1L;
private String charsetName = "UTF-8";

@Override
public void configure(Configuration parameters) {
super.configure(parameters);
this.setDelimiter("</page>");
}

@Override
public String readRecord(String reusable, byte[] bytes, int offset, int numBytes) throws IOException {
return new String(bytes, offset, numBytes, this.charsetName);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
package org.wikipedia.citolytics.edits.operators;

import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.util.Collector;
import org.wikipedia.citolytics.edits.types.ArticleAuthorPair;

import java.util.regex.Matcher;
import java.util.regex.Pattern;

/**
* DataSource format for Wikipedia stub-meta-history.xml dumps
*/
public class EditInputMapper implements FlatMapFunction<String, ArticleAuthorPair> {
@Override
public void flatMap(String xml, Collector<ArticleAuthorPair> out) throws Exception {
// Extract title
Pattern titleRegex = Pattern.compile("<title>(.*?)</title>", Pattern.DOTALL);
Pattern revRegex = Pattern.compile("<revision>(.*?)</revision>", Pattern.DOTALL);
Pattern commentRegex = Pattern.compile("<comment>(.*?)</comment>", Pattern.DOTALL);
Pattern contributorRegex = Pattern.compile("<contributor>(\\s+)<username>(.*?)</username>(\\s+)<id>(.*?)</id>(\\s+)</contributor>", Pattern.DOTALL);

Matcher titleMatcher = titleRegex.matcher(xml);

if (!titleMatcher.find()) {
// no title found
return;
}

String title = titleMatcher.group(1);

Matcher revMatcher = revRegex.matcher(xml);
while (revMatcher.find()) {
Matcher contributorMatcher = contributorRegex.matcher(revMatcher.group(1));
Matcher commentMatcher = commentRegex.matcher(revMatcher.group(1));

String comment = commentMatcher.find() ? commentMatcher.group(1) : "";

if (comment.contains("bot") || comment.contains("Bot") || comment.contains("BOT")) {
// Exclude bot edits
return;
}

if (contributorMatcher.find()) {
// Use name or id?
String authorName = contributorMatcher.group(2);
int authorId = Integer.valueOf(contributorMatcher.group(4));

if (authorId > 0) {
// Exclude guest users
out.collect(new ArticleAuthorPair(title, authorId));
}
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
package org.wikipedia.citolytics.edits.types;

import org.apache.flink.api.java.tuple.Tuple2;

public class ArticleAuthorPair extends Tuple2<String, Integer> {
public ArticleAuthorPair() {
// Flink requires empty constructor
}

public ArticleAuthorPair(String article, int author) {
f0 = article;
f1 = author;
}

public String getArticle() {
return f0;
}

public int getAuthor() {
return f1;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
package org.wikipedia.citolytics.edits.types;

import org.apache.flink.api.java.tuple.Tuple2;

import java.util.List;

public class AuthorArticlesList extends Tuple2<Integer, List<String>> {
public AuthorArticlesList() {
// empty constructor
}

public AuthorArticlesList(int authorId, List<String> articles) {
f0 = authorId;
f1 = articles;
}

public List<String> getArticleList() {
return f1;
}
}
24 changes: 24 additions & 0 deletions src/main/java/org/wikipedia/citolytics/edits/types/CoEditList.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
package org.wikipedia.citolytics.edits.types;

import org.apache.flink.api.java.tuple.Tuple2;

import java.util.List;

public class CoEditList extends Tuple2<String, List<String>> {
public CoEditList() {
// empty constructor
}

public CoEditList(String article, List<String> coEdits) {
f0 = article;
f1 = coEdits;
}

public List<String> getList() {
return f1;
}

public String getArticle() {
return f0;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
package org.wikipedia.citolytics.edits.types;

import org.apache.flink.api.java.tuple.Tuple4;

import java.util.List;

public class EditEvaluationResult extends Tuple4<String, List<String>, List<String>, Integer> {
public EditEvaluationResult() {
// Flink requires empty constructor
}

public EditEvaluationResult(String article, List<String> goldStandard, List<String> recommendations, int relevant) {
f0 = article;
f1 = goldStandard;
f2 = recommendations;
f3 = relevant;
}
}
16 changes: 0 additions & 16 deletions src/main/java/org/wikipedia/processing/DocumentProcessor.java
Original file line number Diff line number Diff line change
@@ -1,19 +1,3 @@
/* __
* \ \
* _ _ \ \ ______
* | | | | > \( __ )
* | |_| |/ ^ \| || |
* | ._,_/_/ \_\_||_|
* | |
* |_|
*
* ----------------------------------------------------------------------------
* "THE BEER-WARE LICENSE" (Revision 42):
* <rob ∂ CLABS dot CC> wrote this file. As long as you retain this notice you
* can do whatever you want with this stuff. If we meet some day, and you think
* this stuff is worth it, you can buy me a beer in return.
* ----------------------------------------------------------------------------
*/
package org.wikipedia.processing;

import org.apache.commons.lang.StringUtils;
Expand Down
16 changes: 0 additions & 16 deletions src/main/java/org/wikipedia/processing/types/WikiDocument.java
Original file line number Diff line number Diff line change
@@ -1,19 +1,3 @@
/* __
* \ \
* _ _ \ \ ______
* | | | | > \( __ )
* | |_| |/ ^ \| || |
* | ._,_/_/ \_\_||_|
* | |
* |_|
*
* ----------------------------------------------------------------------------
* "THE BEER-WARE LICENSE" (Revision 42):
* <rob ∂ CLABS dot CC> wrote this file. As long as you retain this notice you
* can do whatever you want with this stuff. If we meet some day, and you think
* this stuff is worth it, you can buy me a beer in return.
* ----------------------------------------------------------------------------
*/
package org.wikipedia.processing.types;

import org.apache.commons.lang.StringUtils;
Expand Down
Loading

0 comments on commit db2a379

Please sign in to comment.