From eafc85d7ac7657f1284746939a851cbb4ce5f7db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 17 Mar 2017 16:32:26 +0100 Subject: [PATCH 01/20] Initial version of spark-based document deduplication. It contains a new version of the clustering mechanism with auto-sizing clusters. --- .../nbactions.xml | 73 ++++ .../deduplication-document-spark-impl/pom.xml | 161 ++++++++ .../CustomOddsCharsKeyGenerator.java | 136 +++++++ .../deduplication/DeduplicateDocuments.scala | 298 +++++++++++++++ .../DocumentWrapperKryoRegistrator.java | 80 ++++ .../merge/AdvancedDuplicatesMerger.java | 349 ++++++++++++++++++ .../deduplication/merge/DuplicatesMerger.java | 13 + .../merge/SimpleDuplicatesMerger.java | 98 +++++ .../CustomOddsCharsKeyGeneratorTest.java | 110 ++++++ .../pom.xml | 35 ++ deduplication-document-spark/pom.xml | 20 + 11 files changed, 1373 insertions(+) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/pom.xml create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java create mode 100644 deduplication-document-spark/deduplication-document-spark-workflow/pom.xml create mode 100644 deduplication-document-spark/pom.xml diff --git a/deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml b/deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml new file mode 100644 index 00000000..b9137ba9 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml @@ -0,0 +1,73 @@ + + + + run + + jar + + + process-classes + org.codehaus.mojo:exec-maven-plugin:1.2.1:exec + + + -classpath %classpath pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + java + + + + debug + + jar + + + process-classes + org.codehaus.mojo:exec-maven-plugin:1.2.1:exec + + + -Xdebug -Xrunjdwp:transport=dt_socket,server=n,address=${jpda.address} -classpath %classpath pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + java + true + + + + profile + + jar + + + process-classes + org.codehaus.mojo:exec-maven-plugin:1.2.1:exec + + + -classpath %classpath pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + java + + + + CUSTOM-scala:run + scala:run + + scala:run + + + + CUSTOM-RunSmall + RunSmall + + scala:run + + + test|test2 + + + + + CUSTOM-clean,build,upload + clean,build,upload + + clean + install + wagon:upload-single + + + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml new file mode 100644 index 00000000..6461abb8 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -0,0 +1,161 @@ + + + 4.0.0 + + pl.edu.icm.coansys + deduplication-document-spark + 1.11-SNAPSHOT + + + deduplication-document-spark-impl + jar + Deduplication - Document - SparkVersion - Implementation + + + + ssh-cypisek + scpexe://cypisek/jobs + + + + + src/main/scala + src/test/scala + + + + + net.alchim31.maven + scala-maven-plugin + + + + + compile + testCompile + + + + + -dependencyfile + ${project.build.directory}/.scala_dependencies + + + + + + + + base + + pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-shade-plugin + 2.3 + + + package + + shade + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + junit:junit + log4j:log4j:jar: + org.scala-lang:scala-library:jar: + org.apache.spark:spark-core_2.10 + org.apache.spark:spark-sql_2.10 + org.apache.spark:spark-streaming_2.10 + + + ${project.artifactId}-${project.version} + + + + org.codehaus.mojo + wagon-maven-plugin + 1.0-beta-3 + + ${project.build.directory}/${project.build.finalName}.jar + scp://cypisek-gw.ocean.icm.edu.pl/home/axnow/jobs/ + dedupdocs.jar + + + + + + + + + + org.apache.maven.wagon + wagon-ssh + 2.8 + + + + + + + + ${project.groupId} + models + ${project.version} + + + ${project.groupId} + deduplication-document-impl + ${project.version} + + + org.apache.spark + spark-core_2.10 + + + org.apache.spark + spark-graphx_2.10 + + + javax.servlet + javax.servlet-api + 3.1.0 + runtime + + + com.google.guava + guava + 15.0 + + + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java new file mode 100644 index 00000000..442208ff --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java @@ -0,0 +1,136 @@ +/* + * This file is part of CoAnSys project. + * Copyright (c) 2012-2015 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.Arrays; +import pl.edu.icm.coansys.commons.java.DocumentWrapperUtils; +import pl.edu.icm.coansys.commons.java.StringTools; +import pl.edu.icm.coansys.deduplication.document.keygenerator.WorkKeyGenerator; +import pl.edu.icm.coansys.models.DocumentProtos; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; + +/** + * An early deduplication phase key used to group works into clusters. This is + * an extended version, which allows to define requested key size + * + * @author Łukasz Dumiszewski + * @author Aleksander Nowiński + * + */ +public class CustomOddsCharsKeyGenerator implements WorkKeyGenerator { + + public static final int DEFAULT_KEY_SIZE = 5; + + private int keySize = DEFAULT_KEY_SIZE; + + private int[] keySizes = {DEFAULT_KEY_SIZE}; + + public CustomOddsCharsKeyGenerator() { + } + + public CustomOddsCharsKeyGenerator(int keySize) { + this.keySize = keySize; + } + + /** + * Generates reasonable deduplication key for the given + * {@link DocumentWrapper}. The key has size as defined by keySize param. + * The key is created using title of the work, with removed punctuation and + * basic english stopwords, and then keySize odd letters are taken. The + * method is thread-safe. + * + * @param doc document, which key is generated for + * @return an reasonable key for starting deduplication of size keySize + */ + @Override + public String generateKey(DocumentProtos.DocumentMetadata doc) { + String title = DocumentWrapperUtils.getMainTitle(doc); + return generateKey(title); + } + + /** + * Generate a collection of keys of predefined sizes. + * + * @param doc + * @return + */ + public String[] generateKeyList(DocumentProtos.DocumentMetadata doc) { + String title = DocumentWrapperUtils.getMainTitle(doc); + return generateKeys(title); + } + + protected String[] generateKeys(String title) { + title = cleanUpString(title); + String[] res = new String[keySizes.length]; + for (int k = 0; k < keySizes.length; k++) { + int kl = keySizes[k]; + StringBuilder oddCharsSB = new StringBuilder(); + for (int i = 0; i < title.length() && oddCharsSB.length() < kl; i += 2) { + oddCharsSB.append(title.charAt(i)); + } + res[k] = oddCharsSB.toString(); + } + return res; + } + + protected String generateKey(String title) { + title = cleanUpString(title); + + StringBuilder oddCharsSB = new StringBuilder(); + for (int i = 0; i < title.length() && oddCharsSB.length() < keySize; i += 2) { + oddCharsSB.append(title.charAt(i)); + } + return oddCharsSB.toString(); + } + + protected String cleanUpString(String title) { + title = StringTools.normalize(title); //fixme: it seems that normalize, despite javadocs has stopword removal already + title = StringTools.removeStopWords(title); + title = title.replaceAll("\\s", ""); + return title; + } + + public int getKeySize() { + return keySize; + } + + public void setKeySize(int keySize) { + this.keySize = keySize; + } + + public int[] getKeySizes() { + return Arrays.copyOf(keySizes, keySizes.length); + } + + public void setKeySizes(int[] keySizes) { + if (keySizes == null) { + throw new IllegalArgumentException("Null sizes not premitted"); + } + if (keySizes.length < 1) { + throw new IllegalArgumentException("Non empty array required"); + } + for (int i = 0; i < keySizes.length - 1; i++) { + if (keySizes[i] >= keySizes[i + 1]) { + throw new IllegalArgumentException("Array must be sorted in growing order and no equal sizes present."); + } + + } + this.keySizes = Arrays.copyOf(keySizes, keySizes.length); + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala new file mode 100644 index 00000000..72e33920 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -0,0 +1,298 @@ +package pl.edu.icm.coansys.document.deduplication +import scala.collection.JavaConversions._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.hadoop.io.BytesWritable +import org.apache.log4j.Level +import org.apache.log4j.Logger +import org.apache.spark.SparkConf +import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter +import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter +import pl.edu.icm.coansys.deduplication.document.voter.IssueVolumeVoter +import pl.edu.icm.coansys.deduplication.document.voter.JournalVoter +import pl.edu.icm.coansys.deduplication.document.voter.PagesVoter +import pl.edu.icm.coansys.deduplication.document.voter.SimilarityVoter +import pl.edu.icm.coansys.deduplication.document.voter.TitleVoter +import pl.edu.icm.coansys.deduplication.document.voter.YearVoter +import pl.edu.icm.coansys.document.deduplication.merge.AdvancedDuplicatesMerger +import pl.edu.icm.coansys.document.deduplication.merge.DuplicatesMerger +import pl.edu.icm.coansys.models.DocumentProtos +import pl.edu.icm.coansys.models.DocumentProtos._ +import org.apache.spark.rdd.RDD +import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator +import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator +import scala.collection.mutable.ListBuffer + +object DeduplicateDocuments { + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) + val initialClusteringKeySize = 5 + val maximumClusteringKeySize = 15 + val maximumClusterSize = 300 + //max author count - ignored + + // def calculateKey(doc: DocumentMetadata): String = { + // new OddsCharsKeyGenerator().generateKey(doc) + // } + + def calculateKey(doc: DocumentMetadata, size: Int): String = { + new CustomOddsCharsKeyGenerator(size).generateKey(doc) + } + + def prepareClusters(inputDocs: RDD[DocumentWrapper]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Started preparation of clusters.") + var keySize = initialClusteringKeySize + + var approvedClusters: RDD[(String, Iterable[DocumentWrapper])] = null + var docs = inputDocs + var docCount = 0L + var iteration = 1 + + while ({ docCount = docs.count; docCount } > 0 && keySize < maximumClusteringKeySize) { + log.info("Starting iteration %d, keySize: %d, docs to count: %d".format(iteration, keySize, docCount)) + + var processedClusters = docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey() + var posClusters = processedClusters.filter(p => p._2.size <= maximumClusterSize) + if (approvedClusters == null) { + approvedClusters = posClusters + } else { + approvedClusters = approvedClusters.union(posClusters) + } + docs = processedClusters.filter(p => p._2.size > maximumClusterSize).flatMap(p => p._2) + keySize += 1 + iteration += 1 + } + log.info("Finished loop, keySize: %d, iterations done: %d, docs left: %d".format(keySize - 1, iteration - 1, docCount)) + if (docCount > 0) { + log.info("Adding leftovers (%d documents) with keySize %d".format(docCount, keySize)) + approvedClusters = approvedClusters.union(docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey()) + } + approvedClusters + } + + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supported, exitting.") + return ; + } else { + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) + val outputDocuments = args(1) + + // val debugDir = if (args.size > 2) Some(args(2)) else None + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + log.info("Loaded raw bytes.") + // rawbytes.count() + // log.info("Counted raw bytes.") + + val wrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + +// val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() + val initialGroups = prepareClusters(wrappers.map(_._2)) + log.info("After initial group preparation count.") + // val igs = initialGroups.count() + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + // val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) + val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) + val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) + val timing = timedDeduplicated.map(x => x._2) + + log.info("After reducing clusters (comparison)") + val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) + log.info("Finished merge") + // val mergedSize = merged.count() + + val single = initialGroups.filter(t => t._2.size == 1) + // log.info("Got initial group count=" + igs + "; singular=" + single.count + + // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) + // + //now merge the arrays: + val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) + if ("-" != outputDocuments) { + val bas = toWrite.mapValues(doc => doc.toByteArray()) + bas.saveAsSequenceFile(outputDocuments); + } + val tgrouped = timing.groupByKey; + val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect + println("================ Timing stats ======================") + stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) + println("================ end of timing stats ======================") + + println("Exit") + + } + + def buildDocumentsMerger(): DuplicatesMerger = { + val res = new AdvancedDuplicatesMerger + res.setup("") + res + } + + def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { + val merger = buildDocumentsMerger() + val merged = merger.merge(docs); + merged + } + + def buildWorkComparator(): WorkComparator = { + val result = new VotesProductComparator; + result.setMinVotersWeightRequired(1.5f) + result.setProbabilityTreshold(0.5f) + result.setTresholdIncreasingVotersRequired(0.7f) + + val voters = new ListBuffer[SimilarityVoter]() + val dv = new DoiVoter() + dv.setWeight(1.0f) + voters += dv + val jv = new JournalVoter() + jv.setWeight(0.3f) + jv.setDisapproveLevel(0.5f) + jv.setApproveLevel(0.05f) + voters += jv + + val wivv = new IssueVolumeVoter + wivv.setWeight(0.3f) + wivv.setAbstainIfAbsent(true) + wivv.setSubsetResult(0.8f) + wivv.setPartiallyMatchResult(0.52f) + voters += wivv + + val wpv = new PagesVoter + wpv.setWeight(.3f) + wpv.setAbstainIfAbsent(true) + wpv.setAbsentResult(0.6f) + wpv.setSubsetResult(0.75f) + wpv.setPartiallyMatchResult(0.64f) + wpv.setRemoveRepeated(true) + voters += wpv + + val wyv = new YearVoter + wyv.setWeight(.3f) + wyv.setAbstainIfAbsent(true) + wyv.setAbsentResult(.52f) + wyv.setSubsetResult(.9f) + wyv.setPartiallyMatchResult(.75f) + wyv.setRemoveRepeated(true) + voters += wyv + + val wtv = new TitleVoter() + wtv.setWeight(0.8f) + wtv.setDisapproveLevel(0.11f) + wtv.setApproveLevel(0.001f) + wtv.setMaxNormalizedTitleLength(90) + voters += wtv + + val wav = new AuthorsVoter + wav.setWeight(0.8f) + wav.setDisapproveLevel(0.2f) + wav.setApproveLevel(0.03f) + voters += wav + + result.setSimilarityVoters(voters) + result; + } + + //size, num, min time, max time, avg time, avg_group_count + def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { + val count = measurements.size + val times = measurements.map(_._2) + val minTime = times.min / 1000.0 + val maxTime = times.max / 1000.0 + val avgTime = times.sum / (1000.0 * count) + val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) + (size, count, minTime, maxTime, avgTime, avgGroupSize); + } + + def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { + log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) + + val res = Array.ofDim[Int](cluster.size, cluster.size) + val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) + val comparator = buildWorkComparator() + + val sind = sorted.toList.zipWithIndex + + val procStart = System.currentTimeMillis + sind.foreach(p1 => { + val i1 = p1._2 + val d1 = p1._1 + //make sure diagonal is zeroed + (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) + //simple iteration over table + sind.foreach(p2 => { + val i2 = p2._2 + val d2 = p2._1 + if (i1 < i2) { + val s = System.currentTimeMillis + comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) + val e = System.currentTimeMillis + res(i1)(i2) = (e - s).toInt + } + }) + }) + val procEnd = System.currentTimeMillis + val elapsedSteps = res.map(_.sum).sum + log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) + + (sorted, res) + } + + //todo: timing depending on size + //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which + //are assumed to be duplicates. + def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { + log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) + // + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + if (cluster.size > 30) { + + log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) + } + val startTime = System.currentTimeMillis(); + val classes = ListBuffer[ListBuffer[DocumentWrapper]]() + val comparator = buildWorkComparator() + cluster.foreach(x => { + val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) + if (arr.isEmpty) { + val nclass = ListBuffer(x) + classes += nclass + } else { + arr.get += x + } + }) + val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 + val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) + log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") + + res + + } + + def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { + val start = System.currentTimeMillis; + val rresult = deduplicateCluster(cluster, clusterId); + val end = System.currentTimeMillis + + (rresult, (cluster.size, (rresult.size, end - start))) + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java new file mode 100644 index 00000000..726a0bf1 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java @@ -0,0 +1,80 @@ +package pl.edu.icm.coansys.document.deduplication; + + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.spark.serializer.KryoRegistrator; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; + +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +/** + * + * @author Aleksander Nowinski + */ +public class DocumentWrapperKryoRegistrator implements KryoRegistrator { + + @Override + public void registerClasses(Kryo kryo) { + kryo.register(DocumentWrapper.class, new DocumentWrapperSerializer()); + kryo.register(DocumentMetadata.class, new DocumentMetadataSerializer()); + } + + + public static class DocumentWrapperSerializer extends Serializer { + + @Override + public void write(Kryo kryo, Output output, DocumentWrapper object) { + byte[] bytes = object.toByteArray(); + + output.writeInt(bytes.length, true); + output.writeBytes(bytes); + } + + @Override + public DocumentWrapper read(Kryo kryo, Input input, Class type) { + int length = input.readInt(true); + byte[] bytes = input.readBytes(length); + + try { + return DocumentWrapper.parseFrom(bytes);//FIXME: is this exception handling ok? + } catch (InvalidProtocolBufferException ex) { + throw new RuntimeException(ex); + } + } + + } + + public static class DocumentMetadataSerializer extends Serializer { + + @Override + public void write(Kryo kryo, Output output, DocumentMetadata object) { + byte[] bytes = object.toByteArray(); + + output.writeInt(bytes.length, true); + output.writeBytes(bytes); + } + + @Override + public DocumentMetadata read(Kryo kryo, Input input, Class type) { + int length = input.readInt(true); + byte[] bytes = input.readBytes(length); + + try { + return DocumentMetadata.parseFrom(bytes);//FIXME: is this exception handling ok? + } catch (InvalidProtocolBufferException ex) { + throw new RuntimeException(ex); + } + } + + } + +} \ No newline at end of file diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java new file mode 100644 index 00000000..af45ee69 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java @@ -0,0 +1,349 @@ +package pl.edu.icm.coansys.document.deduplication.merge; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.UUID; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.commons.lang.StringUtils; + +import pl.edu.icm.coansys.commons.java.DiacriticsRemover; +import pl.edu.icm.coansys.commons.java.Pair; +import pl.edu.icm.coansys.models.DocumentProtos.Author; +import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import pl.edu.icm.coansys.models.DocumentProtos.KeyValue; +import pl.edu.icm.coansys.models.DocumentProtos.KeywordsList; + +/** + * Class which merge list of DocumentWrapper to one single DocumentWrapper + * object + * + * @author acz + */ +public class AdvancedDuplicatesMerger implements DuplicatesMerger { + public static final String MERGED_ID_SEPARATOR = "+"; + private Map collectionPreferences; + + @Override + public void setup(String collectionPriorities) { + collectionPreferences = new HashMap(); + + for (String coll : collectionPriorities.split("," )) { + coll = coll.trim(); + + Pattern prefPattern = Pattern.compile("^(-?\\d+):(.+)"); + Matcher prefMatcher = prefPattern.matcher(coll); + if (prefMatcher.matches()) { + String priority = prefMatcher.group(1); + String collectionName = prefMatcher.group(2); + collectionPreferences.put(collectionName, Integer.parseInt(priority)); + + } + } + } + + /** + * Chooses the best DocumentWrapper, updates keys in DocumentWrapper, + * DocumentMetadata and authors, gets extIds, auxiliarInfos from all + * DocumentWrappers, matches authors, gets extIds from matched authors + * + * @param duplicates + * @return + */ + @Override + public DocumentWrapper merge(List duplicates) { + + if (duplicates == null || duplicates.isEmpty()) { + throw new RuntimeException("Nothing to merge"); + } else if (duplicates.size() == 1) { + return duplicates.get(0); + } + + int theBestDocumentWrapperIndex = chooseTheBestIndex(duplicates); + + // Collect information from all items + List identifiers = new ArrayList(duplicates.size()); + List allExtIds = new ArrayList(); + List allAuxiliarInfos = new ArrayList(); + SortedSet sortedCollections = new TreeSet(); + List allKeywords = new ArrayList(); + + for (DocumentWrapper dw : duplicates) { + DocumentMetadata dm = dw.getDocumentMetadata(); + identifiers.add(dw.getRowId()); + List collectionList = new ArrayList(dm.getCollectionList()); + if (collectionList.isEmpty()) { + collectionList.add("unknown"); //TODO move to constants + } + for (String collection : collectionList) { + sortedCollections.add(collection); + } + allExtIds.addAll(dm.getExtIdList()); + allAuxiliarInfos.addAll(dm.getAuxiliarInfoList()); + allKeywords.addAll(dm.getKeywordsList()); + } + Collections.sort(identifiers); + String joinedIds = StringUtils.join(identifiers, MERGED_ID_SEPARATOR); + String newIdentifier = UUID.nameUUIDFromBytes(joinedIds.getBytes()).toString(); + + // Create new DocumentWrapper.Builder + DocumentWrapper.Builder resultBuilder = DocumentWrapper + .newBuilder(duplicates.get(theBestDocumentWrapperIndex)); + + // Modify fields of DocumentWrapper.Builder + resultBuilder.setRowId(newIdentifier); + + DocumentMetadata.Builder documentMetadataBuilder = resultBuilder + .getDocumentMetadataBuilder(); + + BasicMetadata.Builder basicMetadataBuilder = documentMetadataBuilder + .getBasicMetadataBuilder(); + documentMetadataBuilder.setKey(newIdentifier); + documentMetadataBuilder.addAllOrigKey(identifiers); + documentMetadataBuilder.addAllCollection(sortedCollections); + + List finalAuthorBuilderList = basicMetadataBuilder + .getAuthorBuilderList(); + for (Author.Builder authorBuilder : finalAuthorBuilderList) { + String positionSuffix = authorBuilder.getKey().replaceAll(".*(#c\\d+)", "$1"); + authorBuilder.setDocId(newIdentifier); + authorBuilder.setKey(newIdentifier + positionSuffix); + } + + List> authorListsToMerge = new ArrayList>(); + + for (int i = 0; i < duplicates.size(); i++) { + if (i != theBestDocumentWrapperIndex) { + List unmatchedList = duplicates.get(i) + .getDocumentMetadata().getBasicMetadata() + .getAuthorList(); + List matchedList = matchAuthors(finalAuthorBuilderList, + unmatchedList); + if (matchedList != null) { + authorListsToMerge.add(matchedList); + } + } + } + + mergeAuthors(finalAuthorBuilderList, authorListsToMerge); + + documentMetadataBuilder.clearExtId(); + documentMetadataBuilder.addAllExtId(mergeKeyValues(allExtIds)); + documentMetadataBuilder + .addAllAuxiliarInfo(mergeKeyValues(allAuxiliarInfos)); + documentMetadataBuilder.addAllKeywords(mergeKeywords(allKeywords)); + + // Build and return DocumentWrapper + return resultBuilder.build(); + } + + /** + * Moves some informations from author lists in listsToMerge to base list + * + * @param base + * @param listsToMerge + */ + protected void mergeAuthors(List base, + List> listsToMerge) { + + for (int i = 0; i < base.size(); i++) { + Author.Builder baseBuilder = base.get(i); + List allExtIds = new ArrayList(); + allExtIds.addAll(baseBuilder.getExtIdList()); + for (List authorsToMerge : listsToMerge) { + Author author = authorsToMerge.get(i); + if (author != null) { + allExtIds.addAll(author.getExtIdList()); + } + } + + baseBuilder.clearExtId(); + baseBuilder.addAllExtId(mergeKeyValues(allExtIds)); + } + } + + /** + * Checks if tho author lists contain the same authors. Returns second list + * in order as in base list. + * + * @param base + * @param second + * @return + */ + protected List matchAuthors(List base, + List second) { + List result = new ArrayList(base.size()); + List secondCopy = new ArrayList(second); + + for (Author.Builder author : base) { + Author foundAuthor = null; + for (Author secondAuthor : secondCopy) { + + if (equalsIgnoreCaseIgnoreDiacritics( + author.getName(), secondAuthor.getName()) + || equalsIgnoreCaseIgnoreDiacritics( + author.getForenames(), secondAuthor.getForenames()) + && equalsIgnoreCaseIgnoreDiacritics( + author.getSurname(), secondAuthor.getSurname())) { + foundAuthor = secondAuthor; + break; + } + } + if (foundAuthor != null) { + result.add(foundAuthor); + secondCopy.remove(foundAuthor); + } else { + result.add(null); + } + } + + if (result.size() == base.size()) { + return result; + } else { + return null; + } + } + + private boolean equalsIgnoreCaseIgnoreDiacritics(String firstName, + String secondName) { + if (firstName.isEmpty() || secondName.isEmpty()) { + return false; + } + return DiacriticsRemover.removeDiacritics(firstName).equalsIgnoreCase( + DiacriticsRemover.removeDiacritics(secondName)); + } + + /** + * Merges KeyValue messages. Removes repetitions, concatenates comments. + * + * @param listWithRepetitions + * @return + */ + protected static List mergeKeyValues(List listWithRepetitions) { + + Map, String> map = new HashMap, String>(); + for (KeyValue extId : listWithRepetitions) { + Pair keyValue = new Pair( + extId.getKey(), extId.getValue()); + String comment = extId.getComment(); + if (!map.containsKey(keyValue)) { + map.put(keyValue, comment); + } else if (!comment.isEmpty()) { + String oldComment = map.get(keyValue); + if (oldComment.isEmpty()) { + map.put(keyValue, comment); + } else { + map.put(keyValue, oldComment + "\t" + comment); + } + } + } + + List result = new ArrayList(); + + for (Map.Entry, String> mapEntry : map.entrySet()) { + KeyValue.Builder kvBuilder = KeyValue.newBuilder(); + kvBuilder.setKey(mapEntry.getKey().getX()); + kvBuilder.setValue(mapEntry.getKey().getY()); + String comment = mapEntry.getValue(); + if (!comment.isEmpty()) { + kvBuilder.setComment(comment); + } + result.add(kvBuilder.build()); + } + + return result; + } + + /** + * Chooses index of item which will be the base for merged result. + * + * @param duplicates + * @return + */ + protected int chooseTheBestIndex(List duplicates) { + if (collectionPreferences == null || collectionPreferences.isEmpty()) { + return 0; + } + + int bestDuplicateIdx = 0; + int bestPref = Integer.MIN_VALUE; + + for (int i = 0; i < duplicates.size(); i++) { + DocumentWrapper dw = duplicates.get(i); + for (String collection : dw.getDocumentMetadata().getCollectionList()) { + + int pref = 0; + if (collectionPreferences.containsKey(collection)) { + pref = collectionPreferences.get(collection); + } + + if (pref > bestPref) { + bestPref = pref; + bestDuplicateIdx = i; + } + } + } + return bestDuplicateIdx; + } + + private List mergeKeywords(List allKeywords) { + Map, Pair, String>> keywordsMap = new HashMap, Pair, String>>(); + // type, lang, keywords, comment + + for (KeywordsList kwdList : allKeywords) { + Pair typeAndLang = new Pair(kwdList.getType(), + kwdList.getLanguage()); + Pair, String> keywordsAndComment; + String comment = kwdList.getComment(); + if (!keywordsMap.containsKey(typeAndLang)) { + keywordsAndComment = new Pair, String>( + new HashSet(), comment); + keywordsMap.put(typeAndLang, keywordsAndComment); + } else { + keywordsAndComment = keywordsMap.get(typeAndLang); + if (!comment.isEmpty()) { + String oldComment = keywordsAndComment.getY(); + if (oldComment.isEmpty()) { + keywordsAndComment.setY(comment); + } else { + keywordsAndComment.setY(oldComment + "\t" + comment); + } + } + } + keywordsAndComment.getX().addAll(kwdList.getKeywordsList()); + } + + List result = new ArrayList(); + for (Map.Entry, Pair, String>> entry : keywordsMap + .entrySet()) { + KeywordsList.Builder kwdlBuilder = KeywordsList.newBuilder(); + String type = entry.getKey().getX(); + String lang = entry.getKey().getY(); + Set keywords = entry.getValue().getX(); + String comment = entry.getValue().getY(); + + if (type != null && !type.isEmpty()) { + kwdlBuilder.setType(type); + } + if (lang != null && !lang.isEmpty()) { + kwdlBuilder.setLanguage(lang); + } + if (comment != null && !comment.isEmpty()) { + kwdlBuilder.setComment(comment); + } + kwdlBuilder.addAllKeywords(keywords); + + result.add(kwdlBuilder.build()); + } + return result; + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java new file mode 100644 index 00000000..dd4327de --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java @@ -0,0 +1,13 @@ +package pl.edu.icm.coansys.document.deduplication.merge; + +import java.util.List; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; + +/** + * + * @author acz + */ +public interface DuplicatesMerger { + public void setup(String mergerConfiguration); + public DocumentWrapper merge(List duplicates); +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java new file mode 100644 index 00000000..2e01d455 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java @@ -0,0 +1,98 @@ +package pl.edu.icm.coansys.document.deduplication.merge; + +//import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.UUID; +import org.apache.commons.lang.StringUtils; +import pl.edu.icm.coansys.models.DocumentProtos.Author; +import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import pl.edu.icm.coansys.models.DocumentProtos.KeyValue; +//import pl.edu.icm.coansys.1output.merge.MergeConstants; + +/** + * Chooses first DocumentWrapper, updates keys in DocumentWrapper, DocumentMetadata and authors, gets extIds from all + * DocumentWrappers + * + * @author acz + */ +public class SimpleDuplicatesMerger implements DuplicatesMerger { + + @Override + public DocumentWrapper merge(List duplicates) { + + if (duplicates == null || duplicates.isEmpty()) { + throw new RuntimeException("Nothing to merge"); + } else if (duplicates.size() == 1) { + return duplicates.get(0); + } + + // Collect information for final result + List identifiers = new ArrayList(duplicates.size()); + Map> extIds = new HashMap>(); + SortedSet sortedCollections = new TreeSet(); + + for (DocumentWrapper dw : duplicates) { + DocumentMetadata dm = dw.getDocumentMetadata(); + identifiers.add(dw.getRowId()); + for (String collection : dm.getCollectionList()) { + sortedCollections.add(collection); + } + for (KeyValue id : dm.getExtIdList()) { + String idSource = id.getKey(); + String idValue = id.getValue(); + if (!extIds.containsKey(idSource)) { + extIds.put(idSource, new HashSet()); + } + extIds.get(idSource).add(idValue); + } + } + Collections.sort(identifiers); + String joinedIds = StringUtils.join(identifiers, "???");//MergeConstants.MERGED_ID_SEPARATOR); + String newIdentifier = UUID.nameUUIDFromBytes(joinedIds.getBytes()).toString(); + + // Create new DocumentWrapper.Builder + DocumentWrapper.Builder resultBuilder = DocumentWrapper.newBuilder(duplicates.get(0)); + + // Modify fields of DocumentWrapper.Builder + resultBuilder.setRowId(newIdentifier); + + DocumentMetadata.Builder documentMetadataBuilder = resultBuilder.getDocumentMetadataBuilder(); + + BasicMetadata.Builder basicMetadataBuilder = documentMetadataBuilder.getBasicMetadataBuilder(); + documentMetadataBuilder.setKey(newIdentifier); + documentMetadataBuilder.addAllCollection(sortedCollections); + + for (Author.Builder authorBuilder : basicMetadataBuilder.getAuthorBuilderList()) { + String positionSuffix = authorBuilder.getKey().replaceAll(".*(#c\\d+)", "$1"); + authorBuilder.setDocId(newIdentifier); + authorBuilder.setKey(newIdentifier + positionSuffix); + } + + documentMetadataBuilder.clearExtId(); + for (String eKey : extIds.keySet()) { + for (String eValue : extIds.get(eKey)) { + KeyValue.Builder ei = KeyValue.newBuilder(); + ei.setKey(eKey); + ei.setValue(eValue); + documentMetadataBuilder.addExtId(ei); + } + } + + // Build and return DocumentWrapper + return resultBuilder.build(); + } + + @Override + public void setup(String mergerConfiguration) { + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java new file mode 100644 index 00000000..5045a14c --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java @@ -0,0 +1,110 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.Arrays; +import static org.testng.Assert.*; +import org.testng.annotations.Test; + +/** + * + * @author Aleksander Nowinski + */ +public class CustomOddsCharsKeyGeneratorTest { + + public CustomOddsCharsKeyGeneratorTest() { + } + + public void setUp() { + } + + public void tearDown() { + } + + /** + * Test of generateKey method, of class CustomOddsCharsKeyGenerator. + */ + @Test + public void testGenerateKey() { + System.out.println("generateKey"); + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + + assertEquals(instance.getKeySize(), CustomOddsCharsKeyGenerator.DEFAULT_KEY_SIZE); + assertEquals(instance.generateKey(""), ""); + assertEquals(instance.generateKey("Short legged cat"), "soteg"); + assertEquals(instance.generateKey("a \t\t12 domino titles"), "1dmnt"); + instance.setKeySize(2); + assertEquals(instance.generateKey("The eleven elves"), "ee"); + assertEquals(instance.generateKey("404"), "44"); + instance.setKeySize(10); + assertEquals(instance.generateKey("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"), "bbbbbbbbbb"); + } + + @Test + public void testSetKeySizes() { + System.out.println("setKeySizes"); + + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + try { + instance.setKeySizes(null); + fail("Permitted null."); + } catch (IllegalArgumentException iae) { + //this is ok. + } + try { + instance.setKeySizes(new int[0]); + fail("Permitted empty array."); + } catch (IllegalArgumentException iae) { + //this is ok. + } + try { + instance.setKeySizes(new int[]{1, 2, 3, 3}); + fail("Permitted two equal params"); + } catch (IllegalArgumentException iae) { + //this is ok. + } + try { + instance.setKeySizes(new int[]{1, 2, 4, 3}); + fail("Permitted unsorted array"); + } catch (IllegalArgumentException iae) { + //this is ok. + } + int[] a = new int[] {1,3,6,7,8,9}; + instance.setKeySizes(a); + assertTrue(Arrays.equals(a, instance.getKeySizes())); + + } + + @Test + public void testGenerateKeys() { + System.out.println("generateKeys"); + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + instance.setKeySizes(new int[]{2, 3, 5}); + assertTrue(Arrays.equals(new String[]{"", "", ""}, instance.generateKeys(""))); + assertTrue(Arrays.equals(new String[]{"so", "sot", "soteg"}, instance.generateKeys("Short legged cat"))); + assertTrue(Arrays.equals(new String[]{"44", "44", "44"}, instance.generateKeys("404"))); + assertTrue(Arrays.equals(new String[]{"bb", "bbb", "bbbbb"}, instance.generateKeys("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"))); + } + + /** + * Test of cleanUpString method, of class CustomOddsCharsKeyGenerator. + */ + @Test + public void testCleanUpString() { + System.out.println("cleanUpString"); + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + assertEquals(instance.cleanUpString("test"), "test"); + assertEquals(instance.cleanUpString(".-- test --:+"), "test"); + assertEquals(instance.cleanUpString(".-- test -ab-:+"), "testab"); + assertEquals(instance.cleanUpString(".-- test 2-ab-:+"), "test2ab"); + assertEquals(instance.cleanUpString("\t\n test \t\ttest "), "testtest"); + assertEquals(instance.cleanUpString("test of cat"), "testcat"); + assertEquals(instance.cleanUpString("TeSt oF caT\t\t\n"), "testcat"); + assertEquals(instance.cleanUpString("Koń jak koń"), "konjakkon"); + assertEquals(instance.cleanUpString(" Litera β"), "literabeta"); + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml new file mode 100644 index 00000000..e1a28320 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml @@ -0,0 +1,35 @@ + + + deduplication-document + pl.edu.icm.coansys + 1.11-SNAPSHOT + + 4.0.0 + deduplication-document-spark-workflow + oozie + Deduplication - Document - SparkVersion - Workflow + + UTF-8 + + + + + pl.edu.icm.maven + oozie-maven-plugin + true + + + + + + pl.edu.icm.coansys + deduplication-document-impl + ${project.version} + + + pl.edu.icm.oozie + oozie-runner + test + + + diff --git a/deduplication-document-spark/pom.xml b/deduplication-document-spark/pom.xml new file mode 100644 index 00000000..b46619a7 --- /dev/null +++ b/deduplication-document-spark/pom.xml @@ -0,0 +1,20 @@ + + + 4.0.0 + + + pl.edu.icm.coansys + coansys + 1.11-SNAPSHOT + + + deduplication-document-spark + pom + Deduplication - Document SparkVersion + http://maven.apache.org + + + deduplication-document-spark-impl + deduplication-document-spark-workflow + + From e95a8cc759f7abd8a38045b5deb5a6053db949b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Tue, 4 Apr 2017 17:43:38 +0200 Subject: [PATCH 02/20] Work on the algorithm which splits large clusters among the cluster to obtain better scalability. --- .../deduplication/DeduplicateDocuments.scala | 328 +++++++++++++----- .../document/deduplication/TileTask.java | 128 +++++++ 2 files changed, 363 insertions(+), 93 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 72e33920..d339b6f1 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -22,123 +22,146 @@ import org.apache.spark.rdd.RDD import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer +import java.io.File +import java.io.PrintWriter +import pl.edu.icm.coansys.document.deduplication._ object DeduplicateDocuments { val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 5 - val maximumClusteringKeySize = 15 - val maximumClusterSize = 300 + val initialClusteringKeySize = 7 + val maximumClusteringKeySize = 18 + val maximumClusterSize = 600 //max author count - ignored // def calculateKey(doc: DocumentMetadata): String = { // new OddsCharsKeyGenerator().generateKey(doc) // } + def isValidDocument(doc: DocumentWrapper): Boolean = { + var res = false + if (doc.hasDocumentMetadata) { + val md = doc.getDocumentMetadata + if(md.hasBasicMetadata) { + val bmd = md.getBasicMetadata + if(bmd.getTitleCount()>0||bmd.getAuthorCount>0||bmd.hasDoi||bmd.hasJournal) { + res=true + } + } + } + res + } + def calculateKey(doc: DocumentMetadata, size: Int): String = { new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - def prepareClusters(inputDocs: RDD[DocumentWrapper]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Started preparation of clusters.") - var keySize = initialClusteringKeySize - - var approvedClusters: RDD[(String, Iterable[DocumentWrapper])] = null - var docs = inputDocs - var docCount = 0L - var iteration = 1 - - while ({ docCount = docs.count; docCount } > 0 && keySize < maximumClusteringKeySize) { - log.info("Starting iteration %d, keySize: %d, docs to count: %d".format(iteration, keySize, docCount)) - - var processedClusters = docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey() - var posClusters = processedClusters.filter(p => p._2.size <= maximumClusterSize) - if (approvedClusters == null) { - approvedClusters = posClusters - } else { - approvedClusters = approvedClusters.union(posClusters) - } - docs = processedClusters.filter(p => p._2.size > maximumClusterSize).flatMap(p => p._2) - keySize += 1 - iteration += 1 - } - log.info("Finished loop, keySize: %d, iterations done: %d, docs left: %d".format(keySize - 1, iteration - 1, docCount)) - if (docCount > 0) { - log.info("Adding leftovers (%d documents) with keySize %d".format(docCount, keySize)) - approvedClusters = approvedClusters.union(docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey()) + def calculateKeys(doc: DocumentMetadata): Array[String] = { + val keySizes = initialClusteringKeySize to maximumClusteringKeySize + val generator = new CustomOddsCharsKeyGenerator() + generator.setKeySizes(keySizes.toArray) + var res = generator.generateKeyList(doc) + //special case: if doc + if (res(0) == "") { + res = Array.fill[String](keySizes.length)(doc.getKey) } - approvedClusters + res } - /** - * @param args the command line arguments - */ - def main(args: Array[String]): Unit = { - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supported, exitting.") - return ; - } else { - println("Arguments:") - args.foreach(println); - } - println("Creating context...") - - val conf = new SparkConf() - .setAppName("Document deduplication") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") - - val sc = new SparkContext(conf) - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) - val outputDocuments = args(1) + def cleanKeyArray(in: Array[String]): Array[String] = { - // val debugDir = if (args.size > 2) Some(args(2)) else None + null + } - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - log.info("Loaded raw bytes.") - // rawbytes.count() - // log.info("Counted raw bytes.") + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Initializing cluster preparation (V2)") + val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) + + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + //build rdd (docId, (clusterId, clusterSize) ) + val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) + //build rdd - (docId, clusterId) + val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { + if (x._2 <= maximumClusterSize) { + if (y._2 <= maximumClusterSize) { + if (x._1.length <= y._1.length) { x } else { y } + } else { + x + } + } else { + if (y._2 <= maximumClusterSize) { + y + } else { + if (x._1.length > y._1.length) { x } else { y } + } + } + }).mapValues(_._1) - val wrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey + } - -// val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() - val initialGroups = prepareClusters(wrappers.map(_._2)) - log.info("After initial group preparation count.") - // val igs = initialGroups.count() - val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - // val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) - val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) - val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) - val timing = timedDeduplicated.map(x => x._2) + def prepareClusters(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Started preparation of clusters.") + var keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + var idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + var approvedClusters: RDD[(String, Iterable[String])] = null; + + //clusters to do are expected to have key as the first in key set. + var clustersToDo = idClusterKeys.map(p => (p._2.head, (p._1, p._2.drop(1)))) + do { + var currentKeySize = keySizes.head + keySizes = keySizes.drop(1) + log.info("Preparing clusters with key size: " + currentKeySize) + val newDivision = clustersToDo.groupByKey + log.info("Created new division, had " + clustersToDo.count + " document to group, built up " + newDivision.count + " clusters out of them.") + def clusterApproved(clusterId: String, cluster: Iterable[(String, Array[String])]): Boolean = { + (cluster.size <= maximumClusterSize) || (clusterId.length < currentKeySize) + } - log.info("After reducing clusters (comparison)") - val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) - log.info("Finished merge") - // val mergedSize = merged.count() + val goodClusters = if (keySizes.length > 0) + newDivision.filter(p => clusterApproved(p._1, p._2)) + else + newDivision - val single = initialGroups.filter(t => t._2.size == 1) - // log.info("Got initial group count=" + igs + "; singular=" + single.count + - // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) - // - //now merge the arrays: - val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) - if ("-" != outputDocuments) { - val bas = toWrite.mapValues(doc => doc.toByteArray()) - bas.saveAsSequenceFile(outputDocuments); - } - val tgrouped = timing.groupByKey; - val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect - println("================ Timing stats ======================") - stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) - println("================ end of timing stats ======================") + val goodClustersCleaned = goodClusters.mapValues(cl => cl.map(_._1)) - println("Exit") + if (approvedClusters == null) { + approvedClusters = goodClustersCleaned + } else { + approvedClusters = approvedClusters.union(goodClustersCleaned) + } + if (keySizes.length > 0) { + val unapprovedClusters = newDivision.filter(p => (!clusterApproved(p._1, p._2))) + clustersToDo = unapprovedClusters.flatMap(p => p._2.map(x => (x._2.head, (x._1, x._2.drop(1))))) + val sleft = clustersToDo.count + log.info("Finished cluster selection, good: " + goodClusters.count + "; too big: " + unapprovedClusters.count + " clusters.") + log.info("Finished iteration, " + sleft + " documents left.") + if (sleft == 0) { + log.info("Cluster preparation finished, all is OK.") + } + } else { + log.info("Finished last iteration.") + } + } while (keySizes.length > 0); + + log.info("Preparing docs again.") + //now join: + val doclClusterMap = approvedClusters.flatMap(v => v._2.map(x => (x, v._1))) + val totalDocs = doclClusterMap.count() + log.info("Finished flat map for join, docs: " + totalDocs) + val clusteringRes = doclClusterMap.join(inputDocs).map(x => x._2); + val clusteringResCount = clusteringRes.count + log.info("Finished join, got clustered docs: " + clusteringResCount) + + val res = clusteringRes.groupByKey() + val resSize = res.count + log.info("Finished cluster preparation, got clusters: " + resSize) + res } def buildDocumentsMerger(): DuplicatesMerger = { @@ -295,4 +318,123 @@ object DeduplicateDocuments { (rresult, (cluster.size, (rresult.size, end - start))) } + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + val enableClusterSummary = false; + val fixInvalidDocuments = true; + val removeDoubles=true; + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supported, exitting.") + return ; + } else { + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + val outputDocuments = args(1) + + // val debugDir = if (args.size > 2) Some(args(2)) else None + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + log.info("Loaded raw bytes.") + // rawbytes.count() + // log.info("Counted raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if(fixInvalidDocuments) { + val x = dirtyWrappers.filter(w=>isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info("Filtering invalid documents done, before filtering: "+preSize+" and after filtering "+afterSize+" documents left.") + x + } else { + dirtyWrappers + } + + val wrappers = if(removeDoubles) { + fixedWrappers.reduceByKey((x,y)=>y) + } else { + fixedWrappers + } + + + // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() + val initialGroups = prepareClustersV2(wrappers) + log.info("After initial group preparation count.") + + if (enableClusterSummary) { + //prepare cluster summary + + val largeClusters = initialGroups.filter(_._2.size > 1000) + + largeClusters.mapValues(_.size).take(250).foreach(println) + + val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect + + // val result = largeClusters.collect() + sampledResult.foreach(x => { + val docs = x._2._2 + val clusterId = x._1 + val fileBase = clusterId + "-%04d-".format(docs.size) + println(fileBase) + + println("Writing docs...") + var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") + docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) + pw.close + }) + return + } + + // val igs = initialGroups.count() + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + //val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) + val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) + val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) + val timing = timedDeduplicated.map(x => x._2) + + log.info("After reducing clusters (comparison)") + val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) + log.info("Finished merge") + // val mergedSize = merged.count() + + val single = initialGroups.filter(t => t._2.size == 1) + // log.info("Got initial group count=" + igs + "; singular=" + single.count + + // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) + // + //now merge the arrays: + val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) + if ("-" != outputDocuments) { + val bas = toWrite.mapValues(doc => doc.toByteArray()) + bas.saveAsSequenceFile(outputDocuments); + } else { + log.info("Simulating timing by counting.") + toWrite.count() + } + val tgrouped = timing.groupByKey; + val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect + println("================ Timing stats ======================") + stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) + println("================ end of timing stats ======================") + + println("Exit") + + } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java new file mode 100644 index 00000000..38887c09 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -0,0 +1,128 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.ListIterator; +import java.util.Set; +import java.util.function.BiPredicate; +import java.util.stream.Collectors; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import scala.Tuple2; + +/** + * + * @author Aleksander Nowinski + */ +public class TileTask { + String clusterId; + List rows; + List columns; + + public TileTask(String clusterId, List rows, List columns) { + this.clusterId = clusterId; + this.rows = rows; + this.columns = columns; + } + + + + public List> processPairs(BiPredicate x) { + List> res = new ArrayList<>(); + + + for (DocumentWrapper row : rows) { + for (DocumentWrapper column : columns) { + if(row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey())<0){ + if(x.test(row, column)) { + res.add(new Tuple2<>(row.getDocumentMetadata().getKey(), column.getDocumentMetadata().getKey())); + } + } + } + } + + return res; + } + + /** Prepare set of tile tasks representing task of comparing the cluster cartesian. It is + * natural to use this as a flatMap operator. + * + * @param clusterId id of the cluster added to the tasks + * @param docs list of documents to be cross-compared + * @param tileSize desired size of the single tile task + * @return list of tasks to be executed in parallel. + */ + public static List parallelize(String clusterId, Collection docs,int tileSize) { + List d = new ArrayList<>(docs); + Collections.sort(d, (d1, d2) -> d1.getDocumentMetadata().getKey().compareTo(d2.getDocumentMetadata().getKey())); + + int ntiles = docs.size()/tileSize+(docs.size()%tileSize==0?0:1); + List portions[] = new List[ntiles]; + for (int i = 0; i < d.size(); i++) { + int idx = i%ntiles; + if(portions[idx]==null) { + portions[idx]=new ArrayList(); + } + portions[idx].add(d.get(i)); + } + List res = new ArrayList<>(); + + + for (int i = 0; i < portions.length; i++) { + List rows = portions[i]; + for (int j = i; j < portions.length; j++) { + List columns = portions[j]; + if(rows!=null && columns!=null) { + res.add(new TileTask(clusterId, rows, columns)); + } + } + + } + return res; + } + + /** + * Method which merges clusters of the identifiers ensuring that afterwards + * all clusters where the same element appears are joined into one. It is + * intended to be used as a reduce operator. + * @param r1 first list of clusters + * @param r2 second list of clusters. + * @return list of clusters. + */ + public static List> coalesceResult(List> r1, List>r2) { + List> all = new ArrayList<>(); + all.addAll(remapToSets(r1)); + all.addAll(remapToSets(r2)); + List> res = new ArrayList<>(); + while(!all.isEmpty()) { + Set current = all.remove(0); + boolean anyChange = false; + do { + anyChange=false; + ListIterator> li = all.listIterator(); + while(li.hasNext()) { + Set next = li.next(); + if(next.stream().anyMatch(f->current.contains(f))) { + current.addAll(next); + li.remove(); + anyChange=true; + } + } + } while(anyChange); //necessary, as there may be chain of induced joins + res.add(new ArrayList(current)); + } + return res; + } + + protected static List> remapToSets(List> r2) { + return r2.stream().map(x->new HashSet(x)).collect(Collectors.toList()); + } + +} From efb3fc2aac51131d9f9395a7aac711159ddfbd0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 7 Apr 2017 15:54:11 +0200 Subject: [PATCH 03/20] Complete version with tiled comparison task. Added programatical logging into stdout, for easier log reading --- .../deduplication-document-spark-impl/pom.xml | 2 - .../deduplication/DeduplicateDocuments.scala | 749 +++++++++--------- .../document/deduplication/TileTask.java | 18 +- .../deduplication/TileTaskNGTest.java | 149 ++++ 4 files changed, 518 insertions(+), 400 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index 6461abb8..6d6263c9 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -21,7 +21,6 @@ src/main/scala src/test/scala - @@ -85,7 +84,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index d339b6f1..67294e0b 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -2,9 +2,12 @@ package pl.edu.icm.coansys.document.deduplication import scala.collection.JavaConversions._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import java.util.function.BiPredicate import org.apache.hadoop.io.BytesWritable +import org.apache.log4j.ConsoleAppender import org.apache.log4j.Level import org.apache.log4j.Logger +import org.apache.log4j.PatternLayout import org.apache.spark.SparkConf import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter @@ -19,422 +22,384 @@ import pl.edu.icm.coansys.document.deduplication.merge.DuplicatesMerger import pl.edu.icm.coansys.models.DocumentProtos import pl.edu.icm.coansys.models.DocumentProtos._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer import java.io.File import java.io.PrintWriter import pl.edu.icm.coansys.document.deduplication._ +import scala.collection.JavaConverters._ object DeduplicateDocuments { - val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 7 - val maximumClusteringKeySize = 18 - val maximumClusterSize = 600 - //max author count - ignored - - // def calculateKey(doc: DocumentMetadata): String = { - // new OddsCharsKeyGenerator().generateKey(doc) - // } - - def isValidDocument(doc: DocumentWrapper): Boolean = { - var res = false - if (doc.hasDocumentMetadata) { - val md = doc.getDocumentMetadata - if(md.hasBasicMetadata) { - val bmd = md.getBasicMetadata - if(bmd.getTitleCount()>0||bmd.getAuthorCount>0||bmd.hasDoi||bmd.hasJournal) { - res=true + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) + val initialClusteringKeySize = 7 + val maximumClusteringKeySize = 18 + val maximumClusterSize = 600 + val tileSize = 50 + + + def isValidDocument(doc: DocumentWrapper): Boolean = { + var res = false + if (doc.hasDocumentMetadata) { + val md = doc.getDocumentMetadata + if (md.hasBasicMetadata) { + val bmd = md.getBasicMetadata + if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { + res = true + } } } + res } - res - } - - def calculateKey(doc: DocumentMetadata, size: Int): String = { - new CustomOddsCharsKeyGenerator(size).generateKey(doc) - } - - def calculateKeys(doc: DocumentMetadata): Array[String] = { - val keySizes = initialClusteringKeySize to maximumClusteringKeySize - val generator = new CustomOddsCharsKeyGenerator() - generator.setKeySizes(keySizes.toArray) - var res = generator.generateKeyList(doc) - //special case: if doc - if (res(0) == "") { - res = Array.fill[String](keySizes.length)(doc.getKey) + + def calculateKey(doc: DocumentMetadata, size: Int): String = { + new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - res - } - - def cleanKeyArray(in: Array[String]): Array[String] = { - - null - } - - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Initializing cluster preparation (V2)") - val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray - log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved - val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) - - val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) - - //build rdd (docId, (clusterId, clusterSize) ) - val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) - //build rdd - (docId, clusterId) - val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { - if (x._2 <= maximumClusterSize) { - if (y._2 <= maximumClusterSize) { - if (x._1.length <= y._1.length) { x } else { y } - } else { - x - } - } else { - if (y._2 <= maximumClusterSize) { - y - } else { - if (x._1.length > y._1.length) { x } else { y } - } - } - }).mapValues(_._1) - - inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey - } - - def prepareClusters(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Started preparation of clusters.") - var keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray - log.info("Will use key sizes: " + keySizes.mkString(", ")) - var idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved - var approvedClusters: RDD[(String, Iterable[String])] = null; - - //clusters to do are expected to have key as the first in key set. - var clustersToDo = idClusterKeys.map(p => (p._2.head, (p._1, p._2.drop(1)))) - do { - var currentKeySize = keySizes.head - keySizes = keySizes.drop(1) - log.info("Preparing clusters with key size: " + currentKeySize) - val newDivision = clustersToDo.groupByKey - log.info("Created new division, had " + clustersToDo.count + " document to group, built up " + newDivision.count + " clusters out of them.") - def clusterApproved(clusterId: String, cluster: Iterable[(String, Array[String])]): Boolean = { - (cluster.size <= maximumClusterSize) || (clusterId.length < currentKeySize) - } - - val goodClusters = if (keySizes.length > 0) - newDivision.filter(p => clusterApproved(p._1, p._2)) - else - newDivision - - val goodClustersCleaned = goodClusters.mapValues(cl => cl.map(_._1)) - - if (approvedClusters == null) { - approvedClusters = goodClustersCleaned - } else { - approvedClusters = approvedClusters.union(goodClustersCleaned) - } - - if (keySizes.length > 0) { - val unapprovedClusters = newDivision.filter(p => (!clusterApproved(p._1, p._2))) - clustersToDo = unapprovedClusters.flatMap(p => p._2.map(x => (x._2.head, (x._1, x._2.drop(1))))) - val sleft = clustersToDo.count - log.info("Finished cluster selection, good: " + goodClusters.count + "; too big: " + unapprovedClusters.count + " clusters.") - log.info("Finished iteration, " + sleft + " documents left.") - if (sleft == 0) { - log.info("Cluster preparation finished, all is OK.") - } - } else { - log.info("Finished last iteration.") - } - } while (keySizes.length > 0); - - log.info("Preparing docs again.") - //now join: - val doclClusterMap = approvedClusters.flatMap(v => v._2.map(x => (x, v._1))) - val totalDocs = doclClusterMap.count() - log.info("Finished flat map for join, docs: " + totalDocs) - val clusteringRes = doclClusterMap.join(inputDocs).map(x => x._2); - val clusteringResCount = clusteringRes.count - log.info("Finished join, got clustered docs: " + clusteringResCount) - - val res = clusteringRes.groupByKey() - val resSize = res.count - log.info("Finished cluster preparation, got clusters: " + resSize) - res - } - - def buildDocumentsMerger(): DuplicatesMerger = { - val res = new AdvancedDuplicatesMerger - res.setup("") - res - } - - def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { - val merger = buildDocumentsMerger() - val merged = merger.merge(docs); - merged - } - - def buildWorkComparator(): WorkComparator = { - val result = new VotesProductComparator; - result.setMinVotersWeightRequired(1.5f) - result.setProbabilityTreshold(0.5f) - result.setTresholdIncreasingVotersRequired(0.7f) - - val voters = new ListBuffer[SimilarityVoter]() - val dv = new DoiVoter() - dv.setWeight(1.0f) - voters += dv - val jv = new JournalVoter() - jv.setWeight(0.3f) - jv.setDisapproveLevel(0.5f) - jv.setApproveLevel(0.05f) - voters += jv - - val wivv = new IssueVolumeVoter - wivv.setWeight(0.3f) - wivv.setAbstainIfAbsent(true) - wivv.setSubsetResult(0.8f) - wivv.setPartiallyMatchResult(0.52f) - voters += wivv - - val wpv = new PagesVoter - wpv.setWeight(.3f) - wpv.setAbstainIfAbsent(true) - wpv.setAbsentResult(0.6f) - wpv.setSubsetResult(0.75f) - wpv.setPartiallyMatchResult(0.64f) - wpv.setRemoveRepeated(true) - voters += wpv - - val wyv = new YearVoter - wyv.setWeight(.3f) - wyv.setAbstainIfAbsent(true) - wyv.setAbsentResult(.52f) - wyv.setSubsetResult(.9f) - wyv.setPartiallyMatchResult(.75f) - wyv.setRemoveRepeated(true) - voters += wyv - - val wtv = new TitleVoter() - wtv.setWeight(0.8f) - wtv.setDisapproveLevel(0.11f) - wtv.setApproveLevel(0.001f) - wtv.setMaxNormalizedTitleLength(90) - voters += wtv - - val wav = new AuthorsVoter - wav.setWeight(0.8f) - wav.setDisapproveLevel(0.2f) - wav.setApproveLevel(0.03f) - voters += wav - - result.setSimilarityVoters(voters) - result; - } - - //size, num, min time, max time, avg time, avg_group_count - def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { - val count = measurements.size - val times = measurements.map(_._2) - val minTime = times.min / 1000.0 - val maxTime = times.max / 1000.0 - val avgTime = times.sum / (1000.0 * count) - val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) - (size, count, minTime, maxTime, avgTime, avgGroupSize); - } - - def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { - log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) - - val res = Array.ofDim[Int](cluster.size, cluster.size) - val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) - val comparator = buildWorkComparator() - - val sind = sorted.toList.zipWithIndex - - val procStart = System.currentTimeMillis - sind.foreach(p1 => { - val i1 = p1._2 - val d1 = p1._1 - //make sure diagonal is zeroed - (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) - //simple iteration over table - sind.foreach(p2 => { - val i2 = p2._2 - val d2 = p2._1 - if (i1 < i2) { - val s = System.currentTimeMillis - comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) - val e = System.currentTimeMillis - res(i1)(i2) = (e - s).toInt + + def calculateKeys(doc: DocumentMetadata): Array[String] = { + val keySizes = initialClusteringKeySize to maximumClusteringKeySize + val generator = new CustomOddsCharsKeyGenerator() + generator.setKeySizes(keySizes.toArray) + var res = generator.generateKeyList(doc) + //special case: if doc + if (res(0) == "") { + res = Array.fill[String](keySizes.length)(doc.getKey) } - }) - }) - val procEnd = System.currentTimeMillis - val elapsedSteps = res.map(_.sum).sum - log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) - - (sorted, res) - } - - //todo: timing depending on size - //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which - //are assumed to be duplicates. - def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { - log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) - // - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - if (cluster.size > 30) { - - log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) + res } - val startTime = System.currentTimeMillis(); - val classes = ListBuffer[ListBuffer[DocumentWrapper]]() - val comparator = buildWorkComparator() - cluster.foreach(x => { - val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) - if (arr.isEmpty) { - val nclass = ListBuffer(x) - classes += nclass - } else { - arr.get += x - } - }) - val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 - val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) - log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") - - res - - } - - def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { - val start = System.currentTimeMillis; - val rresult = deduplicateCluster(cluster, clusterId); - val end = System.currentTimeMillis - - (rresult, (cluster.size, (rresult.size, end - start))) - } - - /** - * @param args the command line arguments - */ - def main(args: Array[String]): Unit = { - val enableClusterSummary = false; - val fixInvalidDocuments = true; - val removeDoubles=true; - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supported, exitting.") - return ; - } else { - println("Arguments:") - args.foreach(println); + + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Initializing cluster preparation (V2)") + val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) + + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + //build rdd (docId, (clusterId, clusterSize) ) + val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) + //build rdd - (docId, clusterId) + val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { + if (x._2 <= maximumClusterSize) { + if (y._2 <= maximumClusterSize) { + if (x._1.length <= y._1.length) { x } else { y } + } else { + x + } + } else { + if (y._2 <= maximumClusterSize) { + y + } else { + if (x._1.length > y._1.length) { x } else { y } + } + } + }).mapValues(_._1) + inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey } - println("Creating context...") - val conf = new SparkConf() - .setAppName("Document deduplication") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + def buildDocumentsMerger(): DuplicatesMerger = { + val res = new AdvancedDuplicatesMerger + res.setup("") + res + } - val sc = new SparkContext(conf) - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" - val outputDocuments = args(1) + def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { + val merger = buildDocumentsMerger() + val merged = merger.merge(docs); + merged + } - // val debugDir = if (args.size > 2) Some(args(2)) else None + def buildWorkComparator(): WorkComparator = { + val result = new VotesProductComparator; + result.setMinVotersWeightRequired(1.5f) + result.setProbabilityTreshold(0.5f) + result.setTresholdIncreasingVotersRequired(0.7f) + + val voters = new ListBuffer[SimilarityVoter]() + val dv = new DoiVoter() + dv.setWeight(1.0f) + voters += dv + val jv = new JournalVoter() + jv.setWeight(0.3f) + jv.setDisapproveLevel(0.5f) + jv.setApproveLevel(0.05f) + voters += jv + + val wivv = new IssueVolumeVoter + wivv.setWeight(0.3f) + wivv.setAbstainIfAbsent(true) + wivv.setSubsetResult(0.8f) + wivv.setPartiallyMatchResult(0.52f) + voters += wivv + + val wpv = new PagesVoter + wpv.setWeight(.3f) + wpv.setAbstainIfAbsent(true) + wpv.setAbsentResult(0.6f) + wpv.setSubsetResult(0.75f) + wpv.setPartiallyMatchResult(0.64f) + wpv.setRemoveRepeated(true) + voters += wpv + + val wyv = new YearVoter + wyv.setWeight(.3f) + wyv.setAbstainIfAbsent(true) + wyv.setAbsentResult(.52f) + wyv.setSubsetResult(.9f) + wyv.setPartiallyMatchResult(.75f) + wyv.setRemoveRepeated(true) + voters += wyv + + val wtv = new TitleVoter() + wtv.setWeight(0.8f) + wtv.setDisapproveLevel(0.11f) + wtv.setApproveLevel(0.001f) + wtv.setMaxNormalizedTitleLength(90) + voters += wtv + + val wav = new AuthorsVoter + wav.setWeight(0.8f) + wav.setDisapproveLevel(0.2f) + wav.setApproveLevel(0.03f) + voters += wav + + result.setSimilarityVoters(voters) + result; + } - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - log.info("Loaded raw bytes.") - // rawbytes.count() - // log.info("Counted raw bytes.") + //size, num, min time, max time, avg time, avg_group_count + def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { + val count = measurements.size + val times = measurements.map(_._2) + val minTime = times.min / 1000.0 + val maxTime = times.max / 1000.0 + val avgTime = times.sum / (1000.0 * count) + val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) + (size, count, minTime, maxTime, avgTime, avgGroupSize); + } - val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) - - //fix invalid documents: - val fixedWrappers = if(fixInvalidDocuments) { - val x = dirtyWrappers.filter(w=>isValidDocument(w._2)) - val afterSize = x.count; - val preSize = dirtyWrappers.count - log.info("Filtering invalid documents done, before filtering: "+preSize+" and after filtering "+afterSize+" documents left.") - x - } else { - dirtyWrappers - } - - val wrappers = if(removeDoubles) { - fixedWrappers.reduceByKey((x,y)=>y) - } else { - fixedWrappers + def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { + log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) + + val res = Array.ofDim[Int](cluster.size, cluster.size) + val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) + val comparator = buildWorkComparator() + + val sind = sorted.toList.zipWithIndex + + val procStart = System.currentTimeMillis + sind.foreach(p1 => { + val i1 = p1._2 + val d1 = p1._1 + //make sure diagonal is zeroed + (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) + //simple iteration over table + sind.foreach(p2 => { + val i2 = p2._2 + val d2 = p2._1 + if (i1 < i2) { + val s = System.currentTimeMillis + comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) + val e = System.currentTimeMillis + res(i1)(i2) = (e - s).toInt + } + }) + }) + val procEnd = System.currentTimeMillis + val elapsedSteps = res.map(_.sum).sum + log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) + + (sorted, res) } - - - // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() - val initialGroups = prepareClustersV2(wrappers) - log.info("After initial group preparation count.") - - if (enableClusterSummary) { - //prepare cluster summary - - val largeClusters = initialGroups.filter(_._2.size > 1000) - - largeClusters.mapValues(_.size).take(250).foreach(println) - - val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect - - // val result = largeClusters.collect() - sampledResult.foreach(x => { - val docs = x._2._2 - val clusterId = x._1 - val fileBase = clusterId + "-%04d-".format(docs.size) - println(fileBase) - - println("Writing docs...") - var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") - docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) - pw.close - }) - return + + //todo: timing depending on size + //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which + //are assumed to be duplicates. + def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { + log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) + // + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + if (cluster.size > 30) { + + log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) + } + val startTime = System.currentTimeMillis(); + val classes = ListBuffer[ListBuffer[DocumentWrapper]]() + val comparator = buildWorkComparator() + cluster.foreach(x => { + val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) + if (arr.isEmpty) { + val nclass = ListBuffer(x) + classes += nclass + } else { + arr.get += x + } + }) + val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 + val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) + log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") + + res + } - // val igs = initialGroups.count() - val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - //val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) - val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) - val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) - val timing = timedDeduplicated.map(x => x._2) - - log.info("After reducing clusters (comparison)") - val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) - log.info("Finished merge") - // val mergedSize = merged.count() - - val single = initialGroups.filter(t => t._2.size == 1) - // log.info("Got initial group count=" + igs + "; singular=" + single.count + - // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) - // - //now merge the arrays: - val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) - if ("-" != outputDocuments) { - val bas = toWrite.mapValues(doc => doc.toByteArray()) - bas.saveAsSequenceFile(outputDocuments); - } else { - log.info("Simulating timing by counting.") - toWrite.count() + def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { + val start = System.currentTimeMillis; + val rresult = deduplicateCluster(cluster, clusterId); + val end = System.currentTimeMillis + + (rresult, (cluster.size, (rresult.size, end - start))) } - val tgrouped = timing.groupByKey; - val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect - println("================ Timing stats ======================") - stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) - println("================ end of timing stats ======================") + implicit def toJavaBiPredicate[A, B](predicate: (A, B) ⇒ Boolean) = + new BiPredicate[A, B] { + def test(a: A, b: B) = predicate(a, b) + } + - println("Exit") + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + val enableClusterSummary = false; + val fixInvalidDocuments = true; + val removeDoubles = true; - } + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supported, exitting.") + return ; + } else { + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender + console.activateOptions(); + Logger.getLogger("pl.edu.icm").addAppender(console) + log.info("Reconfigured logger...") + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + val outputDocuments = args(1) + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + log.info("Loaded raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if (fixInvalidDocuments) { + val x = dirtyWrappers.filter(w => isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info("Filtering invalid documents done, before filtering: " + preSize + " and after filtering " + afterSize + " documents left.") + x + } else { + dirtyWrappers + } + + val wrappers = if (removeDoubles) { + fixedWrappers.reduceByKey((x, y) => y) + } else { + fixedWrappers + } + wrappers.persist(StorageLevel.MEMORY_AND_DISK) + + + // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() + val initialGroups = prepareClustersV2(wrappers) + initialGroups.persist(StorageLevel.MEMORY_AND_DISK) + + if (enableClusterSummary) { + //prepare cluster summary + val largeClusters = initialGroups.filter(_._2.size > 1000) + largeClusters.mapValues(_.size).take(250).foreach(println) + val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect + // val result = largeClusters.collect() + sampledResult.foreach(x => { + val docs = x._2._2 + val clusterId = x._1 + val fileBase = clusterId + "-%04d-".format(docs.size) + println(fileBase) + + println("Writing docs...") + var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") + docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) + pw.close + }) + return + } + + // val igs = initialGroups.count() + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + + val tiledTasks = clustersToDeduplicate.flatMap( + p => TileTask.parallelize(p._1, p._2, tileSize).map(v => (p._1, v)) + ) + + + + val equalityClusters = tiledTasks.flatMapValues( + v => { + val comparator = buildWorkComparator + v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => + comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) + } + ).mapValues(v => List(v._1, v._2)) + val allItemsTraces = initialGroups.flatMapValues(l => l.map(v => List(v.getDocumentMetadata.getKey))) + val finalClusters = equalityClusters.union(allItemsTraces).mapValues(List(_)). + reduceByKey( + (a, b) => TileTask.coalesceResult(a.map(x => x.asJava).asJava, b.map(x => x.asJava).asJava). + asScala.toList.map(x => x.asScala.toList) + ).flatMap( + p => { + val cid = p._1 + val cl = p._2 + cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) + }) + val finalDocClusters = finalClusters.flatMapValues(x => x). + map(v => (v._2, v._1)).join(wrappers).map(v => (v._2._1, v._2._2)). + groupByKey() + + val singularClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size==1) + val pluralClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size>1) + + log.info("After reducing clusters (comparison)") + val merged = pluralClustersAfterDeduplication.map(x => (mergeDocuments(x._2.toList))) + log.info("Finished merge") + // val mergedSize = merged.count() + + val single = initialGroups.filter(t => t._2.size == 1) + // log.info("Got initial group count=" + igs + "; singular=" + single.count + + // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) + // + //now merge the arrays: + val toWrite = merged.union(single.union(singularClustersAfterDeduplication).map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) + if ("-" != outputDocuments) { + val bas = toWrite.mapValues(doc => doc.toByteArray()) + bas.saveAsSequenceFile(outputDocuments); + } else { + log.info("Simulating timing by counting.") + toWrite.count() + } +// val tgrouped = timing.groupByKey; +// val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect +// println("================ Timing stats ======================") +// stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) +// println("================ end of timing stats ======================") + + println("Exit") + + } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java index 38887c09..61caf991 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -5,6 +5,7 @@ */ package pl.edu.icm.coansys.document.deduplication; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -21,7 +22,7 @@ * * @author Aleksander Nowinski */ -public class TileTask { +public class TileTask implements Serializable{ String clusterId; List rows; List columns; @@ -33,15 +34,20 @@ public TileTask(String clusterId, List rows, List> processPairs(BiPredicate x) { + /** + * Generate list of pairs of the documents, where predicate is conformed, ie function + * passed returned true. The predicate is assumed to be symmetrical, so it is executed + * only once on each pair. + * + * @param equalityTest predicate which defines whether or no two elements are considered matching (typically equal) + * @return list of pairs of keys of equal documents (documents where equalityTest returned true) + */ + public List> processPairs(BiPredicate equalityTest) { List> res = new ArrayList<>(); - - for (DocumentWrapper row : rows) { for (DocumentWrapper column : columns) { if(row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey())<0){ - if(x.test(row, column)) { + if(equalityTest.test(row, column)) { res.add(new Tuple2<>(row.getDocumentMetadata().getKey(), column.getDocumentMetadata().getKey())); } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java new file mode 100644 index 00000000..5f67425d --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java @@ -0,0 +1,149 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; +import com.google.common.collect.Range; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.function.BiPredicate; +import java.util.stream.Collectors; +import static org.testng.Assert.*; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import pl.edu.icm.coansys.models.DocumentProtos; +import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import scala.Tuple2; + +/** + * + * @author Aleksander Nowinski + */ +public class TileTaskNGTest { + + public TileTaskNGTest() { + } + + @BeforeMethod + public void setUpMethod() throws Exception { + } + + @AfterMethod + public void tearDownMethod() throws Exception { + } + + public DocumentWrapper createDocument(String key, String title) { + return DocumentWrapper.newBuilder().setDocumentMetadata( + DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( + BasicMetadata.newBuilder().addTitle( + DocumentProtos.TextWithLanguage.newBuilder().setText(title))) + ).setRowId(key).build(); + } + + protected List createDocumentList(int size) { + return ContiguousSet.create(Range.closed(1, size), DiscreteDomain.integers()).stream() + .map(i -> createDocument(String.format("key_%03d", i), String.format("title_%03d", i))) + .collect(Collectors.toList()); + } + + /** + * Test of processPairs method, of class TileTask. + */ + @Test + public void testProcessPairs() { + System.out.println("processPairs"); + int size = 500; + List documents = createDocumentList(size); + System.out.println("Generated documents:"); + System.out.println(documents.stream().limit(10).map(v -> v.toString()).reduce(((s1, s2) -> s1 + "\n" + s2))); + + TileTask instance = new TileTask("test", documents, documents); + List> res = instance.processPairs((a, b) -> false); + assertEquals(res.size(), 0); + + res = instance.processPairs((a, b) -> true); + assertEquals(res.size(), size * (size - 1) / 2); + + res = instance.processPairs( + (a, b) -> { + String k1 = a.getDocumentMetadata().getKey(); + String k2 = b.getDocumentMetadata().getKey(); + String k1s = k1.substring(5); + String k2s = k2.substring(5); + return k1s.equals(k2s); + }); +// System.out.println("Generated pairs:"); +// System.out.println(res.stream().map(v -> v.toString()).reduce(((s1, s2) -> s1 + ", " + s2))); + assertEquals(res.size(), 1000); + + } + + + private Set toKeySet(List wrappers) { + return wrappers.stream().map(x->x.getDocumentMetadata().getKey()).collect(Collectors.toSet()); + } + + /** + * Test of parallelize method, of class TileTask. + */ + @Test + public void testParallelize() { + System.out.println("parallelize"); + List docs = createDocumentList(9); + String clusterId = "cluster"; + List res = TileTask.parallelize(clusterId, docs, 200); + assertEquals(res.size(), 1, "Created too many tasks."); + + res = TileTask.parallelize(clusterId, docs, 5); + assertEquals(res.size(), 3, "Created invalid number of tasks."); + Set tile0r = toKeySet(res.get(0).rows); + Set tile0c = toKeySet(res.get(0).columns); + assertEquals(tile0r, tile0c); + + Set tile1r = toKeySet(res.get(1).rows); + Set tile1c = toKeySet(res.get(1).columns); + tile1r.stream().forEach( + key->assertFalse(tile1c.contains(key),"In tile 1 key "+key+" from row appears in columns.") + ); + + res = TileTask.parallelize(clusterId, docs, 2); + assertEquals(res.size(), 15, "Created invalid number tasks."); + res = TileTask.parallelize(clusterId, docs, 1); + assertEquals(res.size(), 45, "Created too many tasks."); + } + + /** + * Test of coalesceResult method, of class TileTask. + */ + @Test + public void testCoalesceResult() { + System.out.println("coalesceResult"); + List> r1 = Arrays.asList(new List[] { + Arrays.asList(new String[]{"a", "b"}), + Arrays.asList(new String[]{"c", "d"}), + Arrays.asList(new String[]{"e", "f"}), + }); + List> r2 = Arrays.asList(new List[] { + Arrays.asList(new String[]{"a", "c"}), + }); + + List> result = TileTask.coalesceResult(r1, r2); + assertEquals(2, result.size()); + List l1 = result.get(0); + Collections.sort(l1); + assertEquals(l1, + Arrays.asList("a", "b", "c", "d")); + } + + +} From fab0915a16d9f28eaca0ec8f69a1a2c0f75d59c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 14 Apr 2017 12:10:41 +0200 Subject: [PATCH 04/20] Work on tiled optimalization. Added reshuffle for better work balance. --- .../deduplication/DeduplicateDocuments.scala | 104 ++++++++--- .../ExecuteOnceOnExecutorsHelpers.java | 40 ++++ .../document/deduplication/TileTask.java | 173 ++++++++++++------ .../deduplication/TileTaskNGTest.java | 27 ++- 4 files changed, 254 insertions(+), 90 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 67294e0b..cb696062 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -8,7 +8,9 @@ import org.apache.log4j.ConsoleAppender import org.apache.log4j.Level import org.apache.log4j.Logger import org.apache.log4j.PatternLayout +import org.apache.spark.HashPartitioner import org.apache.spark.SparkConf +import org.apache.spark.api.java.function.VoidFunction import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter import pl.edu.icm.coansys.deduplication.document.voter.IssueVolumeVoter @@ -36,7 +38,7 @@ object DeduplicateDocuments { val initialClusteringKeySize = 7 val maximumClusteringKeySize = 18 val maximumClusterSize = 600 - val tileSize = 50 + val tileSize = 30 def isValidDocument(doc: DocumentWrapper): Boolean = { @@ -252,12 +254,33 @@ object DeduplicateDocuments { (rresult, (cluster.size, (rresult.size, end - start))) } - implicit def toJavaBiPredicate[A, B](predicate: (A, B) ⇒ Boolean) = + + implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = new BiPredicate[A, B] { def test(a: A, b: B) = predicate(a, b) } - - + +// +// implicit def toVoidFunction[T](runMethod: (T) => Unit) = +// new VoidFunction[T]() { +// def call(t:T) = runMethod(t) +// } + + + def reconfigureLogging(sc: SparkContext) = { + ExecuteOnceOnExecutorsHelpers.executeEverywhere(sc, "RECONFIGURE_LOGGING", true, + new VoidFunction[java.util.Iterator[_]]() { + def call(t :java.util.Iterator[_]) :Unit={ + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender + console.activateOptions(); + Logger.getLogger("pl.edu.icm").addAppender(console) + log.info("Reconfigured logger...") + } + } + ) + } + /** * @param args the command line arguments */ @@ -283,11 +306,8 @@ object DeduplicateDocuments { .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") val sc = new SparkContext(conf) - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender - console.activateOptions(); - Logger.getLogger("pl.edu.icm").addAppender(console) - log.info("Reconfigured logger...") + reconfigureLogging(sc) + println("Created context...") sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" @@ -303,7 +323,7 @@ object DeduplicateDocuments { val x = dirtyWrappers.filter(w => isValidDocument(w._2)) val afterSize = x.count; val preSize = dirtyWrappers.count - log.info("Filtering invalid documents done, before filtering: " + preSize + " and after filtering " + afterSize + " documents left.") + log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") x } else { dirtyWrappers @@ -315,11 +335,12 @@ object DeduplicateDocuments { fixedWrappers } wrappers.persist(StorageLevel.MEMORY_AND_DISK) + val initialSize = wrappers.count + log.info(f"Starting processing with $initialSize documents.") - - // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() val initialGroups = prepareClustersV2(wrappers) initialGroups.persist(StorageLevel.MEMORY_AND_DISK) + if (enableClusterSummary) { //prepare cluster summary @@ -343,31 +364,54 @@ object DeduplicateDocuments { // val igs = initialGroups.count() val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - - val tiledTasks = clustersToDeduplicate.flatMap( - p => TileTask.parallelize(p._1, p._2, tileSize).map(v => (p._1, v)) - ) - - - - val equalityClusters = tiledTasks.flatMapValues( - v => { - val comparator = buildWorkComparator - v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => - comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) - } - ).mapValues(v => List(v._1, v._2)) - val allItemsTraces = initialGroups.flatMapValues(l => l.map(v => List(v.getDocumentMetadata.getKey))) - val finalClusters = equalityClusters.union(allItemsTraces).mapValues(List(_)). + val initialClusterCount = clustersToDeduplicate.count + + val tiledTasks = clustersToDeduplicate.flatMap( p => TileTask.parallelize(p._1, p._2, tileSize) ) + val shuffledTiledTasks = tiledTasks.repartition(5000); + shuffledTiledTasks.persist(StorageLevel.MEMORY_AND_DISK) + val tileCount = shuffledTiledTasks.count; + + log.info(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") +// val comparator = sc.broadcast(buildWorkComparator) + reconfigureLogging(sc) + val partialEqualityClusters = shuffledTiledTasks.flatMap( + v => { + val t0=java.lang.System.currentTimeMillis() + log.info("Starting tile task %s".format(v.getTaskId)) + val comparator = buildWorkComparator + val res = v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => + comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) + val time = (java.lang.System.currentTimeMillis()-t0)/1000. + log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") + res + } + ) +// equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) +// log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); +// val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) +// log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); +// +// val eqclsizes= filteredSizes.collect.sortBy (_._2) +// println("Equality cluster sizes:") +// eqclsizes.foreach(println(_)) +// println("Done.\n\n") + + val finalClusters = partialEqualityClusters.mapValues(List(_)). reduceByKey( - (a, b) => TileTask.coalesceResult(a.map(x => x.asJava).asJava, b.map(x => x.asJava).asJava). - asScala.toList.map(x => x.asScala.toList) + (a, b) => TileTask.coalesceResult(a, b).toList ).flatMap( p => { val cid = p._1 val cl = p._2 cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) }) + + finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); + val finclSizes=finalClusters.mapValues(_.size).filter(_._2>500).collect.sortBy(_._1) + println("\n\nFinal cluster sizes:") + finclSizes.foreach(println(_)) + println("Done.") + val finalDocClusters = finalClusters.flatMapValues(x => x). map(v => (v._2, v._1)).join(wrappers).map(v => (v._2._1, v._2._2)). groupByKey() diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java new file mode 100644 index 00000000..a413ad20 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java @@ -0,0 +1,40 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.VoidFunction; + +/** + * + * @author Aleksander Nowinski + */ +public class ExecuteOnceOnExecutorsHelpers { + + private static Set executedCode = new HashSet<>(); + + public static void executeEverywhere(SparkContext sc, String codeId, boolean runOnDriver, VoidFunction function) throws Exception{ + JavaSparkContext.fromSparkContext(sc).parallelize(Arrays.asList(new Integer[1000])).foreachPartition(new VoidFunction>() { + @Override + public void call(Iterator it) throws Exception { + if (!executedCode.contains(codeId)) { + ExecuteOnceOnExecutorsHelpers.executedCode.add(codeId); + function.call(it); + } + } + }); + if (runOnDriver) { + function.call(null); + } + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java index 61caf991..4af0524e 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -12,6 +12,7 @@ import java.util.HashSet; import java.util.List; import java.util.ListIterator; +import java.util.Objects; import java.util.Set; import java.util.function.BiPredicate; import java.util.stream.Collectors; @@ -22,8 +23,10 @@ * * @author Aleksander Nowinski */ -public class TileTask implements Serializable{ +public class TileTask implements Serializable { + String clusterId; + String taskId; List rows; List columns; @@ -32,103 +35,165 @@ public TileTask(String clusterId, List rows, List> processPairs(BiPredicate equalityTest) { - List> res = new ArrayList<>(); + * Generate list of pairs of the documents, where predicate is conformed, ie + * function passed returned true. The predicate is assumed to be + * symmetrical, so it is executed only once on each pair. + * + * @param equalityTest predicate which defines whether or no two elements + * are considered matching (typically equal) + * @return list of pairs of keys of equal documents (documents where + * equalityTest returned true), wrappet into a tuple with cluster id. + */ + public List>> processPairs(BiPredicate equalityTest) { + List> raw = new ArrayList<>(); for (DocumentWrapper row : rows) { - for (DocumentWrapper column : columns) { - if(row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey())<0){ - if(equalityTest.test(row, column)) { - res.add(new Tuple2<>(row.getDocumentMetadata().getKey(), column.getDocumentMetadata().getKey())); - } - } - } + final List rlist = new ArrayList<>(); + rlist.add(row.getDocumentMetadata().getKey()); + List equalColums = columns.stream() + .filter( + column -> (row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey()) < 0) + && (equalityTest.test(row, column)) + ).map(column -> column.getDocumentMetadata().getKey()) + .collect(Collectors.toList()); + rlist.addAll(equalColums); + raw.add(rlist); } - - return res; + raw = coalesceResult(raw); + return raw.stream().map(list -> new Tuple2<>(clusterId, list)).collect(Collectors.toList()); } - - /** Prepare set of tile tasks representing task of comparing the cluster cartesian. It is - * natural to use this as a flatMap operator. - * + + /** + * Prepare set of tile tasks representing task of comparing the cluster + * cartesian. It is natural to use this as a flatMap operator. + * * @param clusterId id of the cluster added to the tasks * @param docs list of documents to be cross-compared * @param tileSize desired size of the single tile task * @return list of tasks to be executed in parallel. */ - public static List parallelize(String clusterId, Collection docs,int tileSize) { + public static List parallelize(String clusterId, Collection docs, int tileSize) { List d = new ArrayList<>(docs); Collections.sort(d, (d1, d2) -> d1.getDocumentMetadata().getKey().compareTo(d2.getDocumentMetadata().getKey())); - - int ntiles = docs.size()/tileSize+(docs.size()%tileSize==0?0:1); + + int ntiles = docs.size() / tileSize + (docs.size() % tileSize == 0 ? 0 : 1); List portions[] = new List[ntiles]; for (int i = 0; i < d.size(); i++) { - int idx = i%ntiles; - if(portions[idx]==null) { - portions[idx]=new ArrayList(); + int idx = i % ntiles; + if (portions[idx] == null) { + portions[idx] = new ArrayList(); } portions[idx].add(d.get(i)); } List res = new ArrayList<>(); - - + for (int i = 0; i < portions.length; i++) { List rows = portions[i]; for (int j = i; j < portions.length; j++) { List columns = portions[j]; - if(rows!=null && columns!=null) { - res.add(new TileTask(clusterId, rows, columns)); + if (rows != null && columns != null) { + final TileTask ntask = new TileTask(clusterId, rows, columns); + ntask.taskId = String.format("%s_%04d:%04d", clusterId, i, j); + res.add(ntask); } } - + } return res; } - + /** - * Method which merges clusters of the identifiers ensuring that afterwards - * all clusters where the same element appears are joined into one. It is - * intended to be used as a reduce operator. - * @param r1 first list of clusters - * @param r2 second list of clusters. - * @return list of clusters. + * Combine clusters which have non-empty intersection, so result will be + * only separate lists. + * + * @param clusters lists to combine + * @return list of the separate clusters, as */ - public static List> coalesceResult(List> r1, List>r2) { + public static List> coalesceResult(List> clusters) { List> all = new ArrayList<>(); - all.addAll(remapToSets(r1)); - all.addAll(remapToSets(r2)); + all.addAll(remapToSets(clusters)); List> res = new ArrayList<>(); - while(!all.isEmpty()) { + while (!all.isEmpty()) { Set current = all.remove(0); boolean anyChange = false; do { - anyChange=false; + anyChange = false; ListIterator> li = all.listIterator(); - while(li.hasNext()) { + while (li.hasNext()) { Set next = li.next(); - if(next.stream().anyMatch(f->current.contains(f))) { + if (next.stream().anyMatch(f -> current.contains(f))) { current.addAll(next); li.remove(); - anyChange=true; + anyChange = true; } } - } while(anyChange); //necessary, as there may be chain of induced joins + } while (anyChange); //necessary, as there may be chain of induced joins res.add(new ArrayList(current)); } return res; } + /** + * Method which merges clusters of the identifiers ensuring that afterwards + * all clusters where the same element appears are joined into one. It is + * intended to be used as a reduce operator. + * + * @param r1 first list of clusters + * @param r2 second list of clusters. + * @return list of clusters. + */ + public static List> coalesceResult(List> r1, List> r2) { + List> all = new ArrayList<>(); + all.addAll(r1); + all.addAll(r2); + return coalesceResult(all); + } + protected static List> remapToSets(List> r2) { - return r2.stream().map(x->new HashSet(x)).collect(Collectors.toList()); + return r2.stream().map(x -> new HashSet<>(x)).collect(Collectors.toList()); } - + } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java index 5f67425d..9b3ebd49 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java @@ -9,11 +9,9 @@ import com.google.common.collect.DiscreteDomain; import com.google.common.collect.Range; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Set; -import java.util.function.BiPredicate; import java.util.stream.Collectors; import static org.testng.Assert.*; import org.testng.annotations.AfterMethod; @@ -68,11 +66,20 @@ public void testProcessPairs() { System.out.println(documents.stream().limit(10).map(v -> v.toString()).reduce(((s1, s2) -> s1 + "\n" + s2))); TileTask instance = new TileTask("test", documents, documents); - List> res = instance.processPairs((a, b) -> false); - assertEquals(res.size(), 0); + List>> res = instance.processPairs((a, b) -> false); + + assertEquals(res.size(), size); + assertTrue(res.stream().anyMatch(p->p._2.size()==1)); + assertEquals(res.stream().map(p->p._2.get(0)).distinct().count(), size); + res = instance.processPairs((a, b) -> true); - assertEquals(res.size(), size * (size - 1) / 2); + assertEquals(res.size(), 1); + assertEquals(res.get(0)._2.size(), size); + + + + assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); res = instance.processPairs( (a, b) -> { @@ -84,7 +91,15 @@ public void testProcessPairs() { }); // System.out.println("Generated pairs:"); // System.out.println(res.stream().map(v -> v.toString()).reduce(((s1, s2) -> s1 + ", " + s2))); - assertEquals(res.size(), 1000); + assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); + assertEquals(res.size(), 100); + + res.stream().map(x->x._2).forEach( + x->{ + String ke = x.get(0).substring(5); + assertTrue(x.stream().allMatch(key->key.endsWith(ke))); + } + ); } From 6180bbabb970708441e3dc91d74e2c1a45a302d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 23 Jun 2017 10:34:16 +0200 Subject: [PATCH 05/20] Stable version, does proper job within 2.5h on full data set. Needs code cleanup and qality assurance. --- .../deduplication/DeduplicateDocuments.scala | 700 ++++++++---------- .../ExecuteOnceOnExecutorsHelpers.java | 40 - .../document/deduplication/TileTask.java | 11 +- .../deduplication/TileTaskNGTest.java | 3 +- 4 files changed, 304 insertions(+), 450 deletions(-) delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index cb696062..bedc0857 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -4,13 +4,7 @@ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import java.util.function.BiPredicate import org.apache.hadoop.io.BytesWritable -import org.apache.log4j.ConsoleAppender -import org.apache.log4j.Level -import org.apache.log4j.Logger -import org.apache.log4j.PatternLayout -import org.apache.spark.HashPartitioner import org.apache.spark.SparkConf -import org.apache.spark.api.java.function.VoidFunction import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter import pl.edu.icm.coansys.deduplication.document.voter.IssueVolumeVoter @@ -28,422 +22,320 @@ import org.apache.spark.storage.StorageLevel import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer -import java.io.File -import java.io.PrintWriter import pl.edu.icm.coansys.document.deduplication._ import scala.collection.JavaConverters._ object DeduplicateDocuments { - val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 7 - val maximumClusteringKeySize = 18 - val maximumClusterSize = 600 - val tileSize = 30 - - - def isValidDocument(doc: DocumentWrapper): Boolean = { - var res = false - if (doc.hasDocumentMetadata) { - val md = doc.getDocumentMetadata - if (md.hasBasicMetadata) { - val bmd = md.getBasicMetadata - if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { - res = true - } - } - } - res - } - - def calculateKey(doc: DocumentMetadata, size: Int): String = { - new CustomOddsCharsKeyGenerator(size).generateKey(doc) - } - - def calculateKeys(doc: DocumentMetadata): Array[String] = { - val keySizes = initialClusteringKeySize to maximumClusteringKeySize - val generator = new CustomOddsCharsKeyGenerator() - generator.setKeySizes(keySizes.toArray) - var res = generator.generateKeyList(doc) - //special case: if doc - if (res(0) == "") { - res = Array.fill[String](keySizes.length)(doc.getKey) - } - res - } - - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Initializing cluster preparation (V2)") - val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray - log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved - val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) - - val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) - - //build rdd (docId, (clusterId, clusterSize) ) - val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) - //build rdd - (docId, clusterId) - val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { - if (x._2 <= maximumClusterSize) { - if (y._2 <= maximumClusterSize) { - if (x._1.length <= y._1.length) { x } else { y } - } else { - x - } - } else { - if (y._2 <= maximumClusterSize) { - y - } else { - if (x._1.length > y._1.length) { x } else { y } - } - } - }).mapValues(_._1) - inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey - } - - def buildDocumentsMerger(): DuplicatesMerger = { - val res = new AdvancedDuplicatesMerger - res.setup("") - res - } - - def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { - val merger = buildDocumentsMerger() - val merged = merger.merge(docs); - merged - } - - def buildWorkComparator(): WorkComparator = { - val result = new VotesProductComparator; - result.setMinVotersWeightRequired(1.5f) - result.setProbabilityTreshold(0.5f) - result.setTresholdIncreasingVotersRequired(0.7f) - - val voters = new ListBuffer[SimilarityVoter]() - val dv = new DoiVoter() - dv.setWeight(1.0f) - voters += dv - val jv = new JournalVoter() - jv.setWeight(0.3f) - jv.setDisapproveLevel(0.5f) - jv.setApproveLevel(0.05f) - voters += jv - - val wivv = new IssueVolumeVoter - wivv.setWeight(0.3f) - wivv.setAbstainIfAbsent(true) - wivv.setSubsetResult(0.8f) - wivv.setPartiallyMatchResult(0.52f) - voters += wivv - - val wpv = new PagesVoter - wpv.setWeight(.3f) - wpv.setAbstainIfAbsent(true) - wpv.setAbsentResult(0.6f) - wpv.setSubsetResult(0.75f) - wpv.setPartiallyMatchResult(0.64f) - wpv.setRemoveRepeated(true) - voters += wpv - - val wyv = new YearVoter - wyv.setWeight(.3f) - wyv.setAbstainIfAbsent(true) - wyv.setAbsentResult(.52f) - wyv.setSubsetResult(.9f) - wyv.setPartiallyMatchResult(.75f) - wyv.setRemoveRepeated(true) - voters += wyv - - val wtv = new TitleVoter() - wtv.setWeight(0.8f) - wtv.setDisapproveLevel(0.11f) - wtv.setApproveLevel(0.001f) - wtv.setMaxNormalizedTitleLength(90) - voters += wtv - - val wav = new AuthorsVoter - wav.setWeight(0.8f) - wav.setDisapproveLevel(0.2f) - wav.setApproveLevel(0.03f) - voters += wav - - result.setSimilarityVoters(voters) - result; - } - - //size, num, min time, max time, avg time, avg_group_count - def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { - val count = measurements.size - val times = measurements.map(_._2) - val minTime = times.min / 1000.0 - val maxTime = times.max / 1000.0 - val avgTime = times.sum / (1000.0 * count) - val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) - (size, count, minTime, maxTime, avgTime, avgGroupSize); - } - - def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { - log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) - - val res = Array.ofDim[Int](cluster.size, cluster.size) - val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) - val comparator = buildWorkComparator() - - val sind = sorted.toList.zipWithIndex - - val procStart = System.currentTimeMillis - sind.foreach(p1 => { - val i1 = p1._2 - val d1 = p1._1 - //make sure diagonal is zeroed - (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) - //simple iteration over table - sind.foreach(p2 => { - val i2 = p2._2 - val d2 = p2._1 - if (i1 < i2) { - val s = System.currentTimeMillis - comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) - val e = System.currentTimeMillis - res(i1)(i2) = (e - s).toInt - } - }) - }) - val procEnd = System.currentTimeMillis - val elapsedSteps = res.map(_.sum).sum - log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) - - (sorted, res) - } - - //todo: timing depending on size - //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which - //are assumed to be duplicates. - def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { - log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) - // - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - if (cluster.size > 30) { - - log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) - } - val startTime = System.currentTimeMillis(); - val classes = ListBuffer[ListBuffer[DocumentWrapper]]() - val comparator = buildWorkComparator() - cluster.foreach(x => { - val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) - if (arr.isEmpty) { - val nclass = ListBuffer(x) - classes += nclass - } else { - arr.get += x - } - }) - val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 - val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) - log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") - - res + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) + val initialClusteringKeySize = 7 + val maximumClusteringKeySize = 14 + val maximumClusterSize = 2000 + val tileSize = 100 + + + implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = + new BiPredicate[A, B] { + def test(a: A, b: B) = predicate(a, b) } - def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { - val start = System.currentTimeMillis; - val rresult = deduplicateCluster(cluster, clusterId); - val end = System.currentTimeMillis - - (rresult, (cluster.size, (rresult.size, end - start))) - } - implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = - new BiPredicate[A, B] { - def test(a: A, b: B) = predicate(a, b) + def isValidDocument(doc: DocumentWrapper): Boolean = { //todo: fix based on if return value. + var res = false; + if (doc.hasDocumentMetadata()) { + val md = doc.getDocumentMetadata + if (md.hasBasicMetadata) { + val bmd = md.getBasicMetadata + if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { + res = true } - -// -// implicit def toVoidFunction[T](runMethod: (T) => Unit) = -// new VoidFunction[T]() { -// def call(t:T) = runMethod(t) -// } - - - def reconfigureLogging(sc: SparkContext) = { - ExecuteOnceOnExecutorsHelpers.executeEverywhere(sc, "RECONFIGURE_LOGGING", true, - new VoidFunction[java.util.Iterator[_]]() { - def call(t :java.util.Iterator[_]) :Unit={ - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender - console.activateOptions(); - Logger.getLogger("pl.edu.icm").addAppender(console) - log.info("Reconfigured logger...") - } - } - ) + } } + res + } - /** - * @param args the command line arguments - */ - def main(args: Array[String]): Unit = { - val enableClusterSummary = false; - val fixInvalidDocuments = true; - val removeDoubles = true; - - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supported, exitting.") - return ; - } else { - println("Arguments:") - args.foreach(println); - } - println("Creating context...") - - val conf = new SparkConf() - .setAppName("Document deduplication") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") - val sc = new SparkContext(conf) - reconfigureLogging(sc) - - println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" - val outputDocuments = args(1) - - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - log.info("Loaded raw bytes.") - - val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) - - //fix invalid documents: - val fixedWrappers = if (fixInvalidDocuments) { - val x = dirtyWrappers.filter(w => isValidDocument(w._2)) - val afterSize = x.count; - val preSize = dirtyWrappers.count - log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") - x + def calculateKey(doc: DocumentMetadata, size: Int): String = { + new CustomOddsCharsKeyGenerator(size).generateKey(doc) + } + + def calculateKeys(doc: DocumentMetadata): Array[String] = { + val keySizes = initialClusteringKeySize to maximumClusteringKeySize + val generator = new CustomOddsCharsKeyGenerator() + generator.setKeySizes(keySizes.toArray) + var res = generator.generateKeyList(doc) + //special case: if doc has no title. + if (res(0) == "") { + res = Array.fill[String](keySizes.length)(doc.getKey) + } + res + } + + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Initializing cluster preparation (V2)") + val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) + + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + //build rdd (docId, (clusterId, clusterSize) ) + val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) + //build rdd - (docId, clusterId) + val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { + if (x._2 <= maximumClusterSize) { + if (y._2 <= maximumClusterSize) { + if (x._1.length <= y._1.length) { x } else { y } } else { - dirtyWrappers + x } - - val wrappers = if (removeDoubles) { - fixedWrappers.reduceByKey((x, y) => y) + } else { + if (y._2 <= maximumClusterSize) { + y } else { - fixedWrappers + if (x._1.length > y._1.length) { x } else { y } } - wrappers.persist(StorageLevel.MEMORY_AND_DISK) - val initialSize = wrappers.count - log.info(f"Starting processing with $initialSize documents.") - - val initialGroups = prepareClustersV2(wrappers) - initialGroups.persist(StorageLevel.MEMORY_AND_DISK) - + } + }).mapValues(_._1) + inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey + } + + def buildDocumentsMerger(): DuplicatesMerger = { + val res = new AdvancedDuplicatesMerger + res.setup("") + res + } + + def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { + val merger = buildDocumentsMerger() + val merged = merger.merge(docs); + merged + } + + def buildWorkComparator(): WorkComparator = { + val result = new VotesProductComparator; + result.setMinVotersWeightRequired(1.5f) + result.setProbabilityTreshold(0.5f) + result.setTresholdIncreasingVotersRequired(0.7f) + + val voters = new ListBuffer[SimilarityVoter]() + val dv = new DoiVoter() + dv.setWeight(1.0f) + voters += dv + val jv = new JournalVoter() + jv.setWeight(0.3f) + jv.setDisapproveLevel(0.5f) + jv.setApproveLevel(0.05f) + voters += jv + + val wivv = new IssueVolumeVoter + wivv.setWeight(0.3f) + wivv.setAbstainIfAbsent(true) + wivv.setSubsetResult(0.8f) + wivv.setPartiallyMatchResult(0.52f) + voters += wivv + + val wpv = new PagesVoter + wpv.setWeight(.3f) + wpv.setAbstainIfAbsent(true) + wpv.setAbsentResult(0.6f) + wpv.setSubsetResult(0.75f) + wpv.setPartiallyMatchResult(0.64f) + wpv.setRemoveRepeated(true) + voters += wpv + + val wyv = new YearVoter + wyv.setWeight(.3f) + wyv.setAbstainIfAbsent(true) + wyv.setAbsentResult(.52f) + wyv.setSubsetResult(.9f) + wyv.setPartiallyMatchResult(.75f) + wyv.setRemoveRepeated(true) + voters += wyv + + val wtv = new TitleVoter() + wtv.setWeight(0.8f) + wtv.setDisapproveLevel(0.11f) + wtv.setApproveLevel(0.001f) + wtv.setMaxNormalizedTitleLength(90) + voters += wtv + + val wav = new AuthorsVoter + wav.setWeight(0.8f) + wav.setDisapproveLevel(0.2f) + wav.setApproveLevel(0.03f) + voters += wav + + result.setSimilarityVoters(voters) + result; + } + + + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + val enableClusterSummary = false; + val fixInvalidDocuments = true; + val removeDoubles = true; + + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supplied, exitting.") + return ; + } else { //todo: add arguments interpretation. + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) + // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + // "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" + val outputDocuments = args(1) + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + println("Loaded raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if (fixInvalidDocuments) { + val x = dirtyWrappers.filter(w => isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") + x + } else { + dirtyWrappers + } - if (enableClusterSummary) { - //prepare cluster summary - val largeClusters = initialGroups.filter(_._2.size > 1000) - largeClusters.mapValues(_.size).take(250).foreach(println) - val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect - // val result = largeClusters.collect() - sampledResult.foreach(x => { - val docs = x._2._2 - val clusterId = x._1 - val fileBase = clusterId + "-%04d-".format(docs.size) - println(fileBase) - - println("Writing docs...") - var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") - docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) - pw.close - }) - return + val wrappers = if (removeDoubles) { + fixedWrappers.reduceByKey((x, y) => y) + } else { + fixedWrappers + } +// wrappers.persist(StorageLevel.MEMORY_AND_DISK) + + val initialSize = wrappers.count + println(f"Starting processing with $initialSize documents.") + + val initialGroups = prepareClustersV2(wrappers) + initialGroups.persist//(StorageLevel.MEMORY_AND_DISK) + + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + val initialClusterCount = clustersToDeduplicate.count + //TODO: some statistics here on cluster, would be useful. + + val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) + tiledTasks.persist//(StorageLevel.MEMORY_AND_DISK) + val tileCount = tiledTasks.count; + + println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") + // val comparator = sc.broadcast(buildWorkComparator) //todo: can we do comparator broadcast variable? + + val partialEqualityClusters = tiledTasks.flatMap( + v => { + val t0 = java.lang.System.currentTimeMillis() + // log.info("Starting tile task %s".format(v.getTaskId)) + val comparator = buildWorkComparator + val res = v.processPairs((a: DocumentWrapper, b: DocumentWrapper) => + comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) + val time = (java.lang.System.currentTimeMillis() - t0) / 1000.0 + log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") + res + } + ) + // At this moment we have RDD of (String, Seq[String]) (clusterId and list of 'equal' ids inside this cluster + // let's save it: + // partialEqualityClusters.saveAsObjectFile ("hdfs:///user/axnow/intermediate/pec"); + // //later, in other program - watch, you have tuples out of the box :) + // val loadedPec = sc.objectFile[(String, java.util.List[String])]("intermediate/pec") + // val partialEqualityClusters = loadedPec + // + // + // equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) + // log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); + // val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) + // log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); + // + // val eqclsizes= filteredSizes.collect.sortBy (_._2) + // println("Equality cluster sizes:") + // eqclsizes.foreach(println(_)) + // println("Done.\n\n") + + val finalClusters = partialEqualityClusters.mapValues(List(_)). + reduceByKey(_ ++ _). //one long list of lists of ids + map(pair => { + val t0 = java.lang.System.currentTimeMillis() + val res = TileTask.coalesceResult(pair._2.asJava) + val tt = System.currentTimeMillis() - t0; + val clusterSize=pair._2.size + log.info(f"Finished tile coalesce task. (Cluster,time[s], size): ${pair._1}, ${tt / 1000.0}, ${pair._2.size}") + (pair._1, res) + }). + flatMap( //build proper ids for equality clusters. + p => { + val cid = p._1 + val cl = p._2 + cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) } - - // val igs = initialGroups.count() - val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - val initialClusterCount = clustersToDeduplicate.count - - val tiledTasks = clustersToDeduplicate.flatMap( p => TileTask.parallelize(p._1, p._2, tileSize) ) - val shuffledTiledTasks = tiledTasks.repartition(5000); - shuffledTiledTasks.persist(StorageLevel.MEMORY_AND_DISK) - val tileCount = shuffledTiledTasks.count; - - log.info(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") -// val comparator = sc.broadcast(buildWorkComparator) - reconfigureLogging(sc) - val partialEqualityClusters = shuffledTiledTasks.flatMap( - v => { - val t0=java.lang.System.currentTimeMillis() - log.info("Starting tile task %s".format(v.getTaskId)) - val comparator = buildWorkComparator - val res = v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => - comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) - val time = (java.lang.System.currentTimeMillis()-t0)/1000. - log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") - res - } - ) -// equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) -// log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); -// val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) -// log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); -// -// val eqclsizes= filteredSizes.collect.sortBy (_._2) -// println("Equality cluster sizes:") -// eqclsizes.foreach(println(_)) -// println("Done.\n\n") - - val finalClusters = partialEqualityClusters.mapValues(List(_)). - reduceByKey( - (a, b) => TileTask.coalesceResult(a, b).toList - ).flatMap( - p => { - val cid = p._1 - val cl = p._2 - cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) - }) - - finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); - val finclSizes=finalClusters.mapValues(_.size).filter(_._2>500).collect.sortBy(_._1) - println("\n\nFinal cluster sizes:") - finclSizes.foreach(println(_)) - println("Done.") - - val finalDocClusters = finalClusters.flatMapValues(x => x). - map(v => (v._2, v._1)).join(wrappers).map(v => (v._2._1, v._2._2)). - groupByKey() + ) + //now we got all the items in place + finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); + val finclSizes = finalClusters.mapValues(_.size).takeOrdered(100)(Ordering[Int].on(-_._2)) + println("Top 100 cluster sizes:") + finclSizes.foreach(println(_)) + println("-----\n\n") + + //count clusters, documents in clusters and number of documents to be deduced: + val finalClusterCount = finalClusters.count + val documentInFinalClusterCount = finalClusters.map(_._2.size).reduce(_ + _) + val documentRemovedDuringClusteringCount = documentInFinalClusterCount - finalClusterCount + println(f"Finally created $finalClusterCount clusters, containing $documentInFinalClusterCount documents, $documentRemovedDuringClusteringCount documents will be removed.") + + // merge documents + val docIdWithClusterId = finalClusters.flatMapValues(x => x). + map(v => (v._2, v._1)) + val documentWrappersPrepared = wrappers.leftOuterJoin(docIdWithClusterId); + val mergedDocuments = documentWrappersPrepared.filter(_._2._2.isDefined). + map(x => (x._2._2, List(x._2._1))).reduceByKey(_ ++ _). //get lists of cluster documents by cluster id + map(kv => { + val doc = mergeDocuments(kv._2) + (doc.getDocumentMetadata.getKey, doc) + }) + + // documents not touched + val singularDocuments = documentWrappersPrepared.filter(_._2._2.isEmpty).map(x => (x._1, x._2._1)) + + //final result. + val finalResult = singularDocuments.union(mergedDocuments) + finalResult.persist - val singularClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size==1) - val pluralClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size>1) - - log.info("After reducing clusters (comparison)") - val merged = pluralClustersAfterDeduplication.map(x => (mergeDocuments(x._2.toList))) - log.info("Finished merge") - // val mergedSize = merged.count() - - val single = initialGroups.filter(t => t._2.size == 1) - // log.info("Got initial group count=" + igs + "; singular=" + single.count + - // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) - // - //now merge the arrays: - val toWrite = merged.union(single.union(singularClustersAfterDeduplication).map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) - if ("-" != outputDocuments) { - val bas = toWrite.mapValues(doc => doc.toByteArray()) - bas.saveAsSequenceFile(outputDocuments); - } else { - log.info("Simulating timing by counting.") - toWrite.count() - } -// val tgrouped = timing.groupByKey; -// val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect -// println("================ Timing stats ======================") -// stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) -// println("================ end of timing stats ======================") - - println("Exit") - + val finalSize = finalResult.count + println(f"Final counts:\n-----------\n"+ + f" input: $initialSize\n"+ + f" output: $finalSize\n"+ + f" removed: $documentRemovedDuringClusteringCount\n"+ + f" difference: ${initialSize-finalSize-documentRemovedDuringClusteringCount}") + + if ("-" != outputDocuments) { + val bas = finalResult.mapValues(doc => doc.toByteArray()).saveAsSequenceFile(outputDocuments); + } else { + log.info("Simulating timing by counting.") + finalResult.count() + println("Finished counting.") } + } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java deleted file mode 100644 index a413ad20..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; -import org.apache.spark.SparkContext; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.VoidFunction; - -/** - * - * @author Aleksander Nowinski - */ -public class ExecuteOnceOnExecutorsHelpers { - - private static Set executedCode = new HashSet<>(); - - public static void executeEverywhere(SparkContext sc, String codeId, boolean runOnDriver, VoidFunction function) throws Exception{ - JavaSparkContext.fromSparkContext(sc).parallelize(Arrays.asList(new Integer[1000])).foreachPartition(new VoidFunction>() { - @Override - public void call(Iterator it) throws Exception { - if (!executedCode.contains(codeId)) { - ExecuteOnceOnExecutorsHelpers.executedCode.add(codeId); - function.call(it); - } - } - }); - if (runOnDriver) { - function.call(null); - } - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java index 4af0524e..84130424 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -91,16 +91,17 @@ public boolean equals(Object obj) { public List>> processPairs(BiPredicate equalityTest) { List> raw = new ArrayList<>(); for (DocumentWrapper row : rows) { - final List rlist = new ArrayList<>(); - rlist.add(row.getDocumentMetadata().getKey()); List equalColums = columns.stream() .filter( column -> (row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey()) < 0) && (equalityTest.test(row, column)) ).map(column -> column.getDocumentMetadata().getKey()) .collect(Collectors.toList()); - rlist.addAll(equalColums); - raw.add(rlist); + if (!equalColums.isEmpty()) { + final List rlist = new ArrayList<>(equalColums); + rlist.add(row.getDocumentMetadata().getKey()); + raw.add(rlist); + } } raw = coalesceResult(raw); return raw.stream().map(list -> new Tuple2<>(clusterId, list)).collect(Collectors.toList()); @@ -158,7 +159,7 @@ public static List> coalesceResult(List> clusters) { List> res = new ArrayList<>(); while (!all.isEmpty()) { Set current = all.remove(0); - boolean anyChange = false; + boolean anyChange; do { anyChange = false; ListIterator> li = all.listIterator(); diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java index 9b3ebd49..1b5ad200 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java @@ -57,7 +57,8 @@ protected List createDocumentList(int size) { /** * Test of processPairs method, of class TileTask. */ - @Test +// @Test +// @Ignore public void testProcessPairs() { System.out.println("processPairs"); int size = 500; From 28834d264fda9e490e5535716cfc41cbef2ca9a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Mon, 26 Jun 2017 09:38:59 +0200 Subject: [PATCH 06/20] Added options parsing from command line to control app behaviour. Version used for performance testing. --- .../deduplication/DeduplicateDocuments.scala | 114 ++++++++++++------ 1 file changed, 77 insertions(+), 37 deletions(-) diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index bedc0857..8211560b 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -27,19 +27,12 @@ import scala.collection.JavaConverters._ object DeduplicateDocuments { val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 7 - val maximumClusteringKeySize = 14 - val maximumClusterSize = 2000 - val tileSize = 100 - - implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = new BiPredicate[A, B] { def test(a: A, b: B) = predicate(a, b) } - def isValidDocument(doc: DocumentWrapper): Boolean = { //todo: fix based on if return value. var res = false; if (doc.hasDocumentMetadata()) { @@ -53,13 +46,12 @@ object DeduplicateDocuments { } res } - def calculateKey(doc: DocumentMetadata, size: Int): String = { new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - def calculateKeys(doc: DocumentMetadata): Array[String] = { + def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Array[String] = { val keySizes = initialClusteringKeySize to maximumClusteringKeySize val generator = new CustomOddsCharsKeyGenerator() generator.setKeySizes(keySizes.toArray) @@ -71,11 +63,12 @@ object DeduplicateDocuments { res } - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)], initialClusteringKeySize: Int, + maximumClusteringKeySize: Int, maximumClusterSize: Int): RDD[(String, Iterable[DocumentWrapper])] = { log.info("Initializing cluster preparation (V2)") val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)); //we loose documents here, ony ids are preseved val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) @@ -171,24 +164,67 @@ object DeduplicateDocuments { result; } - + case class Config( + inputFile: String = "", + outputFile: String = "", + dumpClusters: Boolean = false, + keySizeMin: Int = 5, + keySizeMax: Int = 10, + clusterSizeMax: Int = 500, + tileSize: Int = 25 + ) /** * @param args the command line arguments */ def main(args: Array[String]): Unit = { - val enableClusterSummary = false; val fixInvalidDocuments = true; val removeDoubles = true; - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supplied, exitting.") - return ; - } else { //todo: add arguments interpretation. - println("Arguments:") - args.foreach(println); + println("Starting document deduplication") + + val parser = new scopt.OptionParser[Config]("CoAnSys Deduplicate Documents") { + head("Deduplicate documents", "0.1") + +// opt[Unit]('c', "dump-clusters").action((x, c) => +// c.copy(dumpClusters = true)).text("dump similarity clusters into debug/clusters hdfs dir during run") + + opt[Int]("cluster-key-min").abbr("kmn").action((x, c) => c.copy(keySizeMin = x)). + validate(x => + if (x >= 2) success + else failure("Value must be >=2")). + text("shortest valid key for cluster, defines pre-clustering. Recommended value more thab 4, minimum 2.") + + opt[Int]("cluster-key-max").abbr("kmx").action((x, c) => c.copy(keySizeMax = x)). + validate(x => + if (x >= 2 && x <= 20) success + else failure("Value must be >=2")). + text("longest valid key for cluster, during pre-clustering. Used to split large clusters. Recommended value more than min, minimum 2, max 20.") + + opt[Int]("cluster-size-max").abbr("cs").action((x, c) => c.copy(clusterSizeMax = x)). + text("Largest acceptable cluster size during preclustering phase. If cluster exceeds algorithm attempts to use longer key if possible. Typically 400+") + + opt[Int]("tile-size").abbr("ts").action((x, c) => c.copy(keySizeMax = x)). + validate(x => + if (x >= 2) success + else failure("Value must be >=2")). + text("Size of the tile tasks used to split large clusters. Min 2, recommended approx 40") + + arg[String]("").required.text("Input sequence file").action((f, c) => c.copy(inputFile = f)) + arg[String]("").optional.text("Output sequence file. If ommited, then no output is written but calculation is done.").action((f, c) => c.copy(outputFile = f)) + note("Blah") + } + + val cfg: Config = parser.parse(args, Config()) match { + case Some(config) => + println(f"Got config:\n${config}") + println(config); + config + case None => + // arguments are bad, error message will have been displayed + println("No config.") + return } + println("Creating context...") val conf = new SparkConf() @@ -199,11 +235,15 @@ object DeduplicateDocuments { val sc = new SparkContext(conf) println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) + // sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = cfg.inputFile // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" // "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" - val outputDocuments = args(1) + val outputDocuments = cfg.outputFile + val tileSize = cfg.tileSize + val initialClusteringKeySize = cfg.keySizeMin + val maximumClusteringKeySize = cfg.keySizeMax + val maximumClusterSize = cfg.clusterSizeMax val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) println("Loaded raw bytes.") @@ -226,20 +266,20 @@ object DeduplicateDocuments { } else { fixedWrappers } -// wrappers.persist(StorageLevel.MEMORY_AND_DISK) + // wrappers.persist(StorageLevel.MEMORY_AND_DISK) val initialSize = wrappers.count println(f"Starting processing with $initialSize documents.") - val initialGroups = prepareClustersV2(wrappers) - initialGroups.persist//(StorageLevel.MEMORY_AND_DISK) + val initialGroups = prepareClustersV2(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) + initialGroups.persist //(StorageLevel.MEMORY_AND_DISK) val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) val initialClusterCount = clustersToDeduplicate.count //TODO: some statistics here on cluster, would be useful. val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) - tiledTasks.persist//(StorageLevel.MEMORY_AND_DISK) + tiledTasks.persist //(StorageLevel.MEMORY_AND_DISK) val tileCount = tiledTasks.count; println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") @@ -281,7 +321,7 @@ object DeduplicateDocuments { val t0 = java.lang.System.currentTimeMillis() val res = TileTask.coalesceResult(pair._2.asJava) val tt = System.currentTimeMillis() - t0; - val clusterSize=pair._2.size + val clusterSize = pair._2.size log.info(f"Finished tile coalesce task. (Cluster,time[s], size): ${pair._1}, ${tt / 1000.0}, ${pair._2.size}") (pair._1, res) }). @@ -322,15 +362,15 @@ object DeduplicateDocuments { //final result. val finalResult = singularDocuments.union(mergedDocuments) finalResult.persist - + val finalSize = finalResult.count - println(f"Final counts:\n-----------\n"+ - f" input: $initialSize\n"+ - f" output: $finalSize\n"+ - f" removed: $documentRemovedDuringClusteringCount\n"+ - f" difference: ${initialSize-finalSize-documentRemovedDuringClusteringCount}") - - if ("-" != outputDocuments) { + println(f"Final counts:\n-----------\n" + + f" input: $initialSize\n" + + f" output: $finalSize\n" + + f" removed: $documentRemovedDuringClusteringCount\n" + + f" difference: ${initialSize - finalSize - documentRemovedDuringClusteringCount}") + + if ("-" != outputDocuments && !outputDocuments.isEmpty) { val bas = finalResult.mapValues(doc => doc.toByteArray()).saveAsSequenceFile(outputDocuments); } else { log.info("Simulating timing by counting.") From 3a749d4a4ed10c2d949b28a1a062f7b6859b2f9b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Tue, 11 Jul 2017 14:40:40 +0200 Subject: [PATCH 07/20] Added dependency for the scopt. --- .../deduplication-document-spark-impl/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index 6d6263c9..36c2f80f 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -155,5 +155,10 @@ guava 15.0 + + com.github.scopt + scopt_2.10 + 3.6.0 + From 0cfdcbb77919963cd8a1578bc1eebfb08a7032e9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 14 Jul 2017 15:36:40 +0200 Subject: [PATCH 08/20] Task tiling class rewritten to scala, with tests. --- .../deduplication-document-spark-impl/pom.xml | 27 ++- .../deduplication/CartesianTaskSplit.scala | 87 ++++++++ .../CustomOddsCharsKeyGenerator.java | 136 ------------ .../deduplication/DeduplicateDocuments.scala | 91 ++++---- .../DocumentWrapperKryoRegistrator.java | 10 +- .../MultiLengthTitleKeyGenerator.scala | 54 +++++ .../document/deduplication/TileTask.java | 200 ------------------ .../CartesianTaskSplitSuite.scala | 88 ++++++++ .../CustomOddsCharsKeyGeneratorTest.java | 110 ---------- .../MultiLengthTitleKeyGeneratorSuite.scala | 67 ++++++ .../deduplication/TileTaskNGTest.java | 165 --------------- 11 files changed, 362 insertions(+), 673 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index 36c2f80f..aa9a9145 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -109,9 +109,24 @@ scp://cypisek-gw.ocean.icm.edu.pl/home/axnow/jobs/ dedupdocs.jar - - - + + + org.scalatest + scalatest-maven-plugin + 1.0 + + ${project.build.directory}/surefire-reports + . + WDF TestSuite.txt + + + + test + + test + + + @@ -160,5 +175,11 @@ scopt_2.10 3.6.0 + + org.scalatest + scalatest_2.10 + 3.0.1 + test + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala new file mode 100644 index 00000000..2bc76c42 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala @@ -0,0 +1,87 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper + +class CartesianTaskSplit( + val clusterId: String, + val taskId: String, + val rows: Seq[DocumentWrapper], + val columns: Seq[DocumentWrapper] +) { + /** + * Generate list of clusters of the documents, where predicate is conformed, ie + * function passed returned true. The predicate is assumed to be + * symmetrical, so it is executed only once on each pair. Note, that as we + * expect that all the tiles will appear within the task, and the comparison + * operator may be expensive, only situations where row key is lesser than + * column key are taken into account + * + * @param equalityTest predicate which defines whether or no two elements + * are considered matching (typically equal) + * @return list of lists of keys of equal documents (documents where + * equalityTest returned true) + */ + def processPairs(equalityTest: (DocumentWrapper, DocumentWrapper) => Boolean): Seq[Seq[String]] = { + return List.empty + + val clusters: Seq[Seq[String]] = rows.map(row => { + val rkey = row.getDocumentMetadata.getKey + val equalColumnKeys = columns.filter(rkey < _.getDocumentMetadata.getKey) + .filter(equalityTest(row, _)) + .map(_.getDocumentMetadata.getKey) + equalColumnKeys :+ rkey + }).filter(_.size > 1) + CartesianTaskSplit.coalesceClusters(clusters) + } + +} + +object CartesianTaskSplit { + /** + * Combine clusters which have non-empty intersection, so result will be + * only separate lists. + * + * @param clusters lists to combine + * @return list of the separate clusters, obtained from merging input clusters + */ + def coalesceClusters(clusters: Seq[Seq[String]]): Seq[Seq[String]] = { + var sets = clusters.map(_.toSet[String]) + var res = List.empty[Set[String]] + while (!sets.isEmpty) { + var current = sets.head + sets = sets.tail + var ps: (Seq[Set[String]], Seq[Set[String]]) = null + do { + ps = sets.partition(_.exists(current.contains(_))) + current +: ps._1.flatMap(x => x) + sets = ps._2 + } while (!ps._1.isEmpty) + res :+ current + } + res.map(_.toSeq) + } + + /** Split one large cluster into parallel tasks of the given size. + */ + def parallelizeCluster(clusterId: String, documents: Iterable[DocumentWrapper], tileSize: Int): Seq[CartesianTaskSplit] = { + println(f"Document count: ${documents.size}, tile size $tileSize") + val ntiles = documents.size/tileSize + (if(documents.size % tileSize>0) 1 else 0) + println(f"ntiles: $ntiles") + + val sdoc = documents.toSeq.sorted(Ordering.by[DocumentWrapper, String](_.getDocumentMetadata.getKey)) + val groupedDocs = sdoc.zipWithIndex.map(docidx => (docidx._2%ntiles, docidx._1)).groupBy[Int](_._1).mapValues(_.map(_._2)) + val res = groupedDocs.flatMap(kv => + groupedDocs.map(kvin => new CartesianTaskSplit( + clusterId, f"${clusterId}_${kv._1}:${kv._2}",kv._2, kvin._2 + ) + ) + ) + res.toSeq + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java deleted file mode 100644 index 442208ff..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * This file is part of CoAnSys project. - * Copyright (c) 2012-2015 ICM-UW - * - * CoAnSys is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - - * CoAnSys is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with CoAnSys. If not, see . - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.util.Arrays; -import pl.edu.icm.coansys.commons.java.DocumentWrapperUtils; -import pl.edu.icm.coansys.commons.java.StringTools; -import pl.edu.icm.coansys.deduplication.document.keygenerator.WorkKeyGenerator; -import pl.edu.icm.coansys.models.DocumentProtos; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; - -/** - * An early deduplication phase key used to group works into clusters. This is - * an extended version, which allows to define requested key size - * - * @author Łukasz Dumiszewski - * @author Aleksander Nowiński - * - */ -public class CustomOddsCharsKeyGenerator implements WorkKeyGenerator { - - public static final int DEFAULT_KEY_SIZE = 5; - - private int keySize = DEFAULT_KEY_SIZE; - - private int[] keySizes = {DEFAULT_KEY_SIZE}; - - public CustomOddsCharsKeyGenerator() { - } - - public CustomOddsCharsKeyGenerator(int keySize) { - this.keySize = keySize; - } - - /** - * Generates reasonable deduplication key for the given - * {@link DocumentWrapper}. The key has size as defined by keySize param. - * The key is created using title of the work, with removed punctuation and - * basic english stopwords, and then keySize odd letters are taken. The - * method is thread-safe. - * - * @param doc document, which key is generated for - * @return an reasonable key for starting deduplication of size keySize - */ - @Override - public String generateKey(DocumentProtos.DocumentMetadata doc) { - String title = DocumentWrapperUtils.getMainTitle(doc); - return generateKey(title); - } - - /** - * Generate a collection of keys of predefined sizes. - * - * @param doc - * @return - */ - public String[] generateKeyList(DocumentProtos.DocumentMetadata doc) { - String title = DocumentWrapperUtils.getMainTitle(doc); - return generateKeys(title); - } - - protected String[] generateKeys(String title) { - title = cleanUpString(title); - String[] res = new String[keySizes.length]; - for (int k = 0; k < keySizes.length; k++) { - int kl = keySizes[k]; - StringBuilder oddCharsSB = new StringBuilder(); - for (int i = 0; i < title.length() && oddCharsSB.length() < kl; i += 2) { - oddCharsSB.append(title.charAt(i)); - } - res[k] = oddCharsSB.toString(); - } - return res; - } - - protected String generateKey(String title) { - title = cleanUpString(title); - - StringBuilder oddCharsSB = new StringBuilder(); - for (int i = 0; i < title.length() && oddCharsSB.length() < keySize; i += 2) { - oddCharsSB.append(title.charAt(i)); - } - return oddCharsSB.toString(); - } - - protected String cleanUpString(String title) { - title = StringTools.normalize(title); //fixme: it seems that normalize, despite javadocs has stopword removal already - title = StringTools.removeStopWords(title); - title = title.replaceAll("\\s", ""); - return title; - } - - public int getKeySize() { - return keySize; - } - - public void setKeySize(int keySize) { - this.keySize = keySize; - } - - public int[] getKeySizes() { - return Arrays.copyOf(keySizes, keySizes.length); - } - - public void setKeySizes(int[] keySizes) { - if (keySizes == null) { - throw new IllegalArgumentException("Null sizes not premitted"); - } - if (keySizes.length < 1) { - throw new IllegalArgumentException("Non empty array required"); - } - for (int i = 0; i < keySizes.length - 1; i++) { - if (keySizes[i] >= keySizes[i + 1]) { - throw new IllegalArgumentException("Array must be sorted in growing order and no equal sizes present."); - } - - } - this.keySizes = Arrays.copyOf(keySizes, keySizes.length); - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 8211560b..378c8e1f 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -34,44 +34,45 @@ object DeduplicateDocuments { } def isValidDocument(doc: DocumentWrapper): Boolean = { //todo: fix based on if return value. - var res = false; if (doc.hasDocumentMetadata()) { val md = doc.getDocumentMetadata if (md.hasBasicMetadata) { val bmd = md.getBasicMetadata - if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { - res = true - } + (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) + } else { + false } + } else { + false } - res - } - - def calculateKey(doc: DocumentMetadata, size: Int): String = { - new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Array[String] = { + def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Seq[String] = { val keySizes = initialClusteringKeySize to maximumClusteringKeySize - val generator = new CustomOddsCharsKeyGenerator() - generator.setKeySizes(keySizes.toArray) - var res = generator.generateKeyList(doc) - //special case: if doc has no title. - if (res(0) == "") { + var res = MultiLengthTitleKeyGenerator.generateKeys(doc)(keySizes) + if (res.head.isEmpty) { res = Array.fill[String](keySizes.length)(doc.getKey) } res } - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)], initialClusteringKeySize: Int, + /** + * Group items into large clusters, within which detailed analysis will be + * held. Items are grouped by keys generated from the normalised titles. + * If the cluster is too big, then longer keys are used, so smaller clusters are + * generated. Treshold is maximumClusterSize. + * + */ + def prepareInitialClustering(inputDocs: RDD[(String, DocumentWrapper)], initialClusteringKeySize: Int, maximumClusteringKeySize: Int, maximumClusterSize: Int): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Initializing cluster preparation (V2)") - val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Initializing cluster preparation.") + val keySizes = initialClusteringKeySize to maximumClusteringKeySize log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)); //we loose documents here, ony ids are preseved - val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) - - val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys( + doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)) //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(kv => kv._2.map(idcluster => (idcluster, kv._1))) // (clusterId => docId) + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) //(clusterId => clusterSize) //build rdd (docId, (clusterId, clusterSize) ) val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) @@ -237,8 +238,9 @@ object DeduplicateDocuments { println("Created context...") // sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) val inputDocuments = cfg.inputFile - // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" - // "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" + // for pasting into console: + // val inputDocuments = "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + // val inputDocuments = "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" val outputDocuments = cfg.outputFile val tileSize = cfg.tileSize val initialClusteringKeySize = cfg.keySizeMin @@ -266,60 +268,45 @@ object DeduplicateDocuments { } else { fixedWrappers } - // wrappers.persist(StorageLevel.MEMORY_AND_DISK) val initialSize = wrappers.count println(f"Starting processing with $initialSize documents.") - val initialGroups = prepareClustersV2(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) + val initialGroups = prepareInitialClustering(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) initialGroups.persist //(StorageLevel.MEMORY_AND_DISK) val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) val initialClusterCount = clustersToDeduplicate.count //TODO: some statistics here on cluster, would be useful. - val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) +// val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) + val tiledTasks = clustersToDeduplicate.flatMap(p => CartesianTaskSplit.parallelizeCluster(p._1, p._2, tileSize)) + tiledTasks.persist //(StorageLevel.MEMORY_AND_DISK) val tileCount = tiledTasks.count; println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") // val comparator = sc.broadcast(buildWorkComparator) //todo: can we do comparator broadcast variable? + //build (clusterId, Seq(docId)) rdd: val partialEqualityClusters = tiledTasks.flatMap( - v => { + task => { val t0 = java.lang.System.currentTimeMillis() - // log.info("Starting tile task %s".format(v.getTaskId)) val comparator = buildWorkComparator - val res = v.processPairs((a: DocumentWrapper, b: DocumentWrapper) => + val res = task.processPairs((a: DocumentWrapper, b: DocumentWrapper) => comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) val time = (java.lang.System.currentTimeMillis() - t0) / 1000.0 - log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") - res + //useful for identification of possible problem. + log.info(f"Finishing tile task ${task.taskId} in $time%.4f sec") + res.map((task.clusterId, _)) } ) - // At this moment we have RDD of (String, Seq[String]) (clusterId and list of 'equal' ids inside this cluster - // let's save it: - // partialEqualityClusters.saveAsObjectFile ("hdfs:///user/axnow/intermediate/pec"); - // //later, in other program - watch, you have tuples out of the box :) - // val loadedPec = sc.objectFile[(String, java.util.List[String])]("intermediate/pec") - // val partialEqualityClusters = loadedPec - // - // - // equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) - // log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); - // val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) - // log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); - // - // val eqclsizes= filteredSizes.collect.sortBy (_._2) - // println("Equality cluster sizes:") - // eqclsizes.foreach(println(_)) - // println("Done.\n\n") - + val finalClusters = partialEqualityClusters.mapValues(List(_)). - reduceByKey(_ ++ _). //one long list of lists of ids + reduceByKey(_ ++ _). //one long list of lists of ids for each cluster map(pair => { val t0 = java.lang.System.currentTimeMillis() - val res = TileTask.coalesceResult(pair._2.asJava) + val res = CartesianTaskSplit.coalesceClusters(pair._2.asJava) val tt = System.currentTimeMillis() - t0; val clusterSize = pair._2.size log.info(f"Finished tile coalesce task. (Cluster,time[s], size): ${pair._1}, ${tt / 1000.0}, ${pair._2.size}") diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java index 726a0bf1..1c096122 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java @@ -10,15 +10,11 @@ import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ /** - * - * @author Aleksander Nowinski + * Simple class which registers custom serializers for the documents protocol + * buffer generated classes. + * @author Aleksander Nowinski */ public class DocumentWrapperKryoRegistrator implements KryoRegistrator { diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala new file mode 100644 index 00000000..24ec2cc2 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala @@ -0,0 +1,54 @@ +/* + * This file is part of CoAnSys project. + * Copyright (c) 2012-2015 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ +package pl.edu.icm.coansys.document.deduplication +import pl.edu.icm.coansys.commons.java.DocumentWrapperUtils +import pl.edu.icm.coansys.commons.java.StringTools +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata + +/** + * Generator for the keys used in early stage of the document deduplication. + */ +class MultiLengthTitleKeyGenerator(val keySizes: Seq[Int]) { + def cleanUpString(title: String): String = { + val normalized = StringTools.normalize(title); + //seems that normalize removes stopwords, which is wrong, and quite expensive + //val normalized = StringTools.removeStopWords(StringTools.normalize(title)); + val res = normalized.replaceAll("\\s+", "") + res + } + + def generateKeys(title: String): Seq[String] = { + val ctitle = cleanUpString(title) + val mlen = keySizes.max + val longestKey = ctitle.zipWithIndex.filter(_._2 % 2 == 0).map(_._1).take(mlen).mkString + keySizes.map(keyLength => longestKey.substring(0, Math.min(keyLength, longestKey.size))).distinct + } + + def generateKeys(document: DocumentMetadata): Seq[String] = { + val title: String = DocumentWrapperUtils.getMainTitle(document) + generateKeys(title) + } +} + + +object MultiLengthTitleKeyGenerator { + def generateKeys(document: DocumentMetadata)(keySizes: Seq[Int]): Seq[String] = { + val generator = new MultiLengthTitleKeyGenerator(keySizes) + generator.generateKeys(document) + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java deleted file mode 100644 index 84130424..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.ListIterator; -import java.util.Objects; -import java.util.Set; -import java.util.function.BiPredicate; -import java.util.stream.Collectors; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; -import scala.Tuple2; - -/** - * - * @author Aleksander Nowinski - */ -public class TileTask implements Serializable { - - String clusterId; - String taskId; - List rows; - List columns; - - public TileTask(String clusterId, List rows, List columns) { - this.clusterId = clusterId; - this.rows = rows; - this.columns = columns; - } - - public String getClusterId() { - return clusterId; - } - - public String getTaskId() { - return taskId; - } - - @Override - public String toString() { - return "TileTask{" + "clusterId=" + clusterId + ", taskId=" + taskId + ", rows[" + rows.size() + "], columns[" + columns.size() + "]}"; - } - - @Override - public int hashCode() { - int hash = 7; - hash = 89 * hash + Objects.hashCode(this.clusterId); - hash = 89 * hash + Objects.hashCode(this.taskId); - return hash; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - final TileTask other = (TileTask) obj; - if (!Objects.equals(this.clusterId, other.clusterId)) { - return false; - } - if (!Objects.equals(this.taskId, other.taskId)) { - return false; - } - return true; - } - - /** - * Generate list of pairs of the documents, where predicate is conformed, ie - * function passed returned true. The predicate is assumed to be - * symmetrical, so it is executed only once on each pair. - * - * @param equalityTest predicate which defines whether or no two elements - * are considered matching (typically equal) - * @return list of pairs of keys of equal documents (documents where - * equalityTest returned true), wrappet into a tuple with cluster id. - */ - public List>> processPairs(BiPredicate equalityTest) { - List> raw = new ArrayList<>(); - for (DocumentWrapper row : rows) { - List equalColums = columns.stream() - .filter( - column -> (row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey()) < 0) - && (equalityTest.test(row, column)) - ).map(column -> column.getDocumentMetadata().getKey()) - .collect(Collectors.toList()); - if (!equalColums.isEmpty()) { - final List rlist = new ArrayList<>(equalColums); - rlist.add(row.getDocumentMetadata().getKey()); - raw.add(rlist); - } - } - raw = coalesceResult(raw); - return raw.stream().map(list -> new Tuple2<>(clusterId, list)).collect(Collectors.toList()); - } - - /** - * Prepare set of tile tasks representing task of comparing the cluster - * cartesian. It is natural to use this as a flatMap operator. - * - * @param clusterId id of the cluster added to the tasks - * @param docs list of documents to be cross-compared - * @param tileSize desired size of the single tile task - * @return list of tasks to be executed in parallel. - */ - public static List parallelize(String clusterId, Collection docs, int tileSize) { - List d = new ArrayList<>(docs); - Collections.sort(d, (d1, d2) -> d1.getDocumentMetadata().getKey().compareTo(d2.getDocumentMetadata().getKey())); - - int ntiles = docs.size() / tileSize + (docs.size() % tileSize == 0 ? 0 : 1); - List portions[] = new List[ntiles]; - for (int i = 0; i < d.size(); i++) { - int idx = i % ntiles; - if (portions[idx] == null) { - portions[idx] = new ArrayList(); - } - portions[idx].add(d.get(i)); - } - List res = new ArrayList<>(); - - for (int i = 0; i < portions.length; i++) { - List rows = portions[i]; - for (int j = i; j < portions.length; j++) { - List columns = portions[j]; - if (rows != null && columns != null) { - final TileTask ntask = new TileTask(clusterId, rows, columns); - ntask.taskId = String.format("%s_%04d:%04d", clusterId, i, j); - res.add(ntask); - } - } - - } - return res; - } - - /** - * Combine clusters which have non-empty intersection, so result will be - * only separate lists. - * - * @param clusters lists to combine - * @return list of the separate clusters, as - */ - public static List> coalesceResult(List> clusters) { - List> all = new ArrayList<>(); - all.addAll(remapToSets(clusters)); - List> res = new ArrayList<>(); - while (!all.isEmpty()) { - Set current = all.remove(0); - boolean anyChange; - do { - anyChange = false; - ListIterator> li = all.listIterator(); - while (li.hasNext()) { - Set next = li.next(); - if (next.stream().anyMatch(f -> current.contains(f))) { - current.addAll(next); - li.remove(); - anyChange = true; - } - } - } while (anyChange); //necessary, as there may be chain of induced joins - res.add(new ArrayList(current)); - } - return res; - } - - /** - * Method which merges clusters of the identifiers ensuring that afterwards - * all clusters where the same element appears are joined into one. It is - * intended to be used as a reduce operator. - * - * @param r1 first list of clusters - * @param r2 second list of clusters. - * @return list of clusters. - */ - public static List> coalesceResult(List> r1, List> r2) { - List> all = new ArrayList<>(); - all.addAll(r1); - all.addAll(r2); - return coalesceResult(all); - } - - protected static List> remapToSets(List> r2) { - return r2.stream().map(x -> new HashSet<>(x)).collect(Collectors.toList()); - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala new file mode 100644 index 00000000..c4e0f191 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala @@ -0,0 +1,88 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import org.scalatest.FunSuite +import org.scalatest._ +import pl.edu.icm.coansys.models.DocumentProtos +import pl.edu.icm.coansys.models.DocumentProtos._ + +class CartesianTaskSplitSuite extends FunSuite with GivenWhenThen { + + + def createDocument(key:String , title:String):DocumentWrapper = { + DocumentProtos.DocumentWrapper.newBuilder().setDocumentMetadata( + DocumentProtos.DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( + DocumentProtos.BasicMetadata.newBuilder().addTitle( + DocumentProtos.TextWithLanguage.newBuilder().setText(title))) + ).setRowId(key).build(); + } + + def createDocumentList(size:Int):Seq[DocumentWrapper] = { + (1 to size).map(idx => createDocument(f"key_$idx", f"title_$idx")).toSeq + } + + def crossProduct[T](l1:Seq[T]):Seq[(T,T)] = { + crossProduct(l1, l1) + } + + + def crossProduct[T](l1:Seq[T], l2:Seq[T]):Seq[(T,T)] = { + l1.flatMap(x1=>l2.map((x1,_))) + } + + + + test("Parallelize empty set") { + Given("Empty task list") + When("We parallelise") + val res = CartesianTaskSplit.parallelizeCluster("testCluster", Seq.empty[DocumentWrapper], 10) + Then("result is empty") + assert(res.isEmpty) + } + + test("Parallelize set") { + Given("Set of 5 documents") + val docs = createDocumentList(5) + val clusterId = "testCluster" + When("We parallelise with size equal to doc number") + val res = CartesianTaskSplit.parallelizeCluster(clusterId, docs, docs.size) + Then("result is single item") + assertResult(1)(res.size) + When("We parallelize with large tile size") + val r2 = CartesianTaskSplit.parallelizeCluster(clusterId, docs, docs.size+3) + Then("result is single item") + assertResult(1)(r2.size) + When("We parallelize with large 3") + val r3 = CartesianTaskSplit.parallelizeCluster(clusterId, docs, 3) + Then("result have 4 tasks") + assertResult(4)(r3.size) + And("Each task the same Given clusterId") + assert(r3.forall(_.clusterId==clusterId)) + } + + + +// +// test("All items present in result tasks") { +// Given("Set of 5 documents") +// val docs = createDocumentList(5) +// val clusterId = "testCluster" +// When("We parallelise to size 2") +// val res = CartesianTaskSplit.parallelizeCluster(clusterId, docs,2) +// Then("Expect 9 tasks") +// assertResult(9)(res.size) +// And("Each cartesian pair is present") +// val allPairs = crossProduct(docs.map(_.getDocumentMetadata.getKey)).toSet +// +// val taskPairs = res.flatMap(task=> {crossProduct(task.rows, task.columns)}).toSet +// assertResult(allPairs)(taskPairs) +// +// } + + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java deleted file mode 100644 index 5045a14c..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.util.Arrays; -import static org.testng.Assert.*; -import org.testng.annotations.Test; - -/** - * - * @author Aleksander Nowinski - */ -public class CustomOddsCharsKeyGeneratorTest { - - public CustomOddsCharsKeyGeneratorTest() { - } - - public void setUp() { - } - - public void tearDown() { - } - - /** - * Test of generateKey method, of class CustomOddsCharsKeyGenerator. - */ - @Test - public void testGenerateKey() { - System.out.println("generateKey"); - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - - assertEquals(instance.getKeySize(), CustomOddsCharsKeyGenerator.DEFAULT_KEY_SIZE); - assertEquals(instance.generateKey(""), ""); - assertEquals(instance.generateKey("Short legged cat"), "soteg"); - assertEquals(instance.generateKey("a \t\t12 domino titles"), "1dmnt"); - instance.setKeySize(2); - assertEquals(instance.generateKey("The eleven elves"), "ee"); - assertEquals(instance.generateKey("404"), "44"); - instance.setKeySize(10); - assertEquals(instance.generateKey("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"), "bbbbbbbbbb"); - } - - @Test - public void testSetKeySizes() { - System.out.println("setKeySizes"); - - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - try { - instance.setKeySizes(null); - fail("Permitted null."); - } catch (IllegalArgumentException iae) { - //this is ok. - } - try { - instance.setKeySizes(new int[0]); - fail("Permitted empty array."); - } catch (IllegalArgumentException iae) { - //this is ok. - } - try { - instance.setKeySizes(new int[]{1, 2, 3, 3}); - fail("Permitted two equal params"); - } catch (IllegalArgumentException iae) { - //this is ok. - } - try { - instance.setKeySizes(new int[]{1, 2, 4, 3}); - fail("Permitted unsorted array"); - } catch (IllegalArgumentException iae) { - //this is ok. - } - int[] a = new int[] {1,3,6,7,8,9}; - instance.setKeySizes(a); - assertTrue(Arrays.equals(a, instance.getKeySizes())); - - } - - @Test - public void testGenerateKeys() { - System.out.println("generateKeys"); - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - instance.setKeySizes(new int[]{2, 3, 5}); - assertTrue(Arrays.equals(new String[]{"", "", ""}, instance.generateKeys(""))); - assertTrue(Arrays.equals(new String[]{"so", "sot", "soteg"}, instance.generateKeys("Short legged cat"))); - assertTrue(Arrays.equals(new String[]{"44", "44", "44"}, instance.generateKeys("404"))); - assertTrue(Arrays.equals(new String[]{"bb", "bbb", "bbbbb"}, instance.generateKeys("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"))); - } - - /** - * Test of cleanUpString method, of class CustomOddsCharsKeyGenerator. - */ - @Test - public void testCleanUpString() { - System.out.println("cleanUpString"); - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - assertEquals(instance.cleanUpString("test"), "test"); - assertEquals(instance.cleanUpString(".-- test --:+"), "test"); - assertEquals(instance.cleanUpString(".-- test -ab-:+"), "testab"); - assertEquals(instance.cleanUpString(".-- test 2-ab-:+"), "test2ab"); - assertEquals(instance.cleanUpString("\t\n test \t\ttest "), "testtest"); - assertEquals(instance.cleanUpString("test of cat"), "testcat"); - assertEquals(instance.cleanUpString("TeSt oF caT\t\t\n"), "testcat"); - assertEquals(instance.cleanUpString("Koń jak koń"), "konjakkon"); - assertEquals(instance.cleanUpString(" Litera β"), "literabeta"); - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala new file mode 100644 index 00000000..13fc490c --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala @@ -0,0 +1,67 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import org.scalatest.FunSuite +import org.scalatest._ + +class MultiLengthTitleKeyGeneratorSuite extends FunSuite with GivenWhenThen { + test("cleaning the string") { + Given("an empty instance") + val instance = new MultiLengthTitleKeyGenerator(3 to 7) + When("empty string is given") + val empty = "" + Then("result should be empty") + assertResult("")(instance.cleanUpString(empty)) + + When("String has varied case") + val varcas = "SomeCaseS" + Then("result should be lowercase") + assertResult("somecases")(instance.cleanUpString(varcas)) + + When("String has spaces") + val spc = "Some spaces" + Then("result should be lowercase, no spaces") + assertResult("somespaces")(instance.cleanUpString(spc)) + + When("String has punctuation") + val pct = "String with \"so called\" - phy - punctuation!" + Then("result have no punctuation nor spaces") + assertResult("stringwithsocalledphypunctuation")(instance.cleanUpString(pct)) + + When("String has some stopwords") + val stopwords = "A the long! of short and tall" + Then("result should contain no stopwords") + assertResult("longshorttall")(instance.cleanUpString(stopwords)) + info("That's all folks!") + } + + test("Building the key set") { + Given("An empty instance with sequence keyset from 1 to 6") + val instance = new MultiLengthTitleKeyGenerator(1 to 6) + When("empty string is given") + val empty = "" + Then("result should be list with single, empty string element.") + assert(instance.generateKeys(empty).size==1) + assert(instance.generateKeys(empty)(0).isEmpty) + + When("Normal string is given") + val normal = "abcdefghijklmnopqr" + Then("result array has appropriate lengths") + val normalRes = instance.generateKeys(normal) + assert(normalRes.map(_.size).toList == (1 to 6).toList) + And("result arrray has proper contents.") + assertResult(List("a", "ac", "ace", "aceg", "acegi", "acegik"))(normalRes.toList) + + When("Short string is given") + val short = "abcdef" + Then("result array has appropriate lengths") + val shortRes = instance.generateKeys(short) + assert(shortRes.map(_.size).toList == (1 to 3).toList) + assertResult(List("a", "ac", "ace"))(shortRes.toList) + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java deleted file mode 100644 index 1b5ad200..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import com.google.common.collect.ContiguousSet; -import com.google.common.collect.DiscreteDomain; -import com.google.common.collect.Range; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import static org.testng.Assert.*; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; -import pl.edu.icm.coansys.models.DocumentProtos; -import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; -import scala.Tuple2; - -/** - * - * @author Aleksander Nowinski - */ -public class TileTaskNGTest { - - public TileTaskNGTest() { - } - - @BeforeMethod - public void setUpMethod() throws Exception { - } - - @AfterMethod - public void tearDownMethod() throws Exception { - } - - public DocumentWrapper createDocument(String key, String title) { - return DocumentWrapper.newBuilder().setDocumentMetadata( - DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( - BasicMetadata.newBuilder().addTitle( - DocumentProtos.TextWithLanguage.newBuilder().setText(title))) - ).setRowId(key).build(); - } - - protected List createDocumentList(int size) { - return ContiguousSet.create(Range.closed(1, size), DiscreteDomain.integers()).stream() - .map(i -> createDocument(String.format("key_%03d", i), String.format("title_%03d", i))) - .collect(Collectors.toList()); - } - - /** - * Test of processPairs method, of class TileTask. - */ -// @Test -// @Ignore - public void testProcessPairs() { - System.out.println("processPairs"); - int size = 500; - List documents = createDocumentList(size); - System.out.println("Generated documents:"); - System.out.println(documents.stream().limit(10).map(v -> v.toString()).reduce(((s1, s2) -> s1 + "\n" + s2))); - - TileTask instance = new TileTask("test", documents, documents); - List>> res = instance.processPairs((a, b) -> false); - - assertEquals(res.size(), size); - assertTrue(res.stream().anyMatch(p->p._2.size()==1)); - assertEquals(res.stream().map(p->p._2.get(0)).distinct().count(), size); - - - res = instance.processPairs((a, b) -> true); - assertEquals(res.size(), 1); - assertEquals(res.get(0)._2.size(), size); - - - - assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); - - res = instance.processPairs( - (a, b) -> { - String k1 = a.getDocumentMetadata().getKey(); - String k2 = b.getDocumentMetadata().getKey(); - String k1s = k1.substring(5); - String k2s = k2.substring(5); - return k1s.equals(k2s); - }); -// System.out.println("Generated pairs:"); -// System.out.println(res.stream().map(v -> v.toString()).reduce(((s1, s2) -> s1 + ", " + s2))); - assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); - assertEquals(res.size(), 100); - - res.stream().map(x->x._2).forEach( - x->{ - String ke = x.get(0).substring(5); - assertTrue(x.stream().allMatch(key->key.endsWith(ke))); - } - ); - - } - - - private Set toKeySet(List wrappers) { - return wrappers.stream().map(x->x.getDocumentMetadata().getKey()).collect(Collectors.toSet()); - } - - /** - * Test of parallelize method, of class TileTask. - */ - @Test - public void testParallelize() { - System.out.println("parallelize"); - List docs = createDocumentList(9); - String clusterId = "cluster"; - List res = TileTask.parallelize(clusterId, docs, 200); - assertEquals(res.size(), 1, "Created too many tasks."); - - res = TileTask.parallelize(clusterId, docs, 5); - assertEquals(res.size(), 3, "Created invalid number of tasks."); - Set tile0r = toKeySet(res.get(0).rows); - Set tile0c = toKeySet(res.get(0).columns); - assertEquals(tile0r, tile0c); - - Set tile1r = toKeySet(res.get(1).rows); - Set tile1c = toKeySet(res.get(1).columns); - tile1r.stream().forEach( - key->assertFalse(tile1c.contains(key),"In tile 1 key "+key+" from row appears in columns.") - ); - - res = TileTask.parallelize(clusterId, docs, 2); - assertEquals(res.size(), 15, "Created invalid number tasks."); - res = TileTask.parallelize(clusterId, docs, 1); - assertEquals(res.size(), 45, "Created too many tasks."); - } - - /** - * Test of coalesceResult method, of class TileTask. - */ - @Test - public void testCoalesceResult() { - System.out.println("coalesceResult"); - List> r1 = Arrays.asList(new List[] { - Arrays.asList(new String[]{"a", "b"}), - Arrays.asList(new String[]{"c", "d"}), - Arrays.asList(new String[]{"e", "f"}), - }); - List> r2 = Arrays.asList(new List[] { - Arrays.asList(new String[]{"a", "c"}), - }); - - List> result = TileTask.coalesceResult(r1, r2); - assertEquals(2, result.size()); - List l1 = result.get(0); - Collections.sort(l1); - assertEquals(l1, - Arrays.asList("a", "b", "c", "d")); - } - - -} From 2e0c205b07a3b73188a0ebf5474791fca5fef20f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Sun, 23 Jul 2017 23:01:44 +0200 Subject: [PATCH 09/20] Scala version. --- .../deduplication-document-spark-impl/pom.xml | 15 +- .../deduplication/CartesianTaskSplit.scala | 28 +++- .../deduplication/DeduplicateDocuments.scala | 158 ++++++++++++------ .../MultiLengthTitleKeyGenerator.scala | 2 +- .../DeduplicateDocumentTest.scala | 124 ++++++++++++++ 5 files changed, 264 insertions(+), 63 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index aa9a9145..4abc9031 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -10,7 +10,12 @@ deduplication-document-spark-impl jar Deduplication - Document - SparkVersion - Implementation - + + + GNU AFFERO GENERAL PUBLIC LICENSE, Version 3 (AGPL-3.0) + http://opensource.org/licenses/AGPL-3.0 + + ssh-cypisek @@ -22,6 +27,8 @@ src/main/scala src/test/scala + + net.alchim31.maven @@ -181,5 +188,11 @@ 3.0.1 test + + com.holdenkarau + spark-testing-base_2.11 + 1.6.0_0.7.2 + test + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala index 2bc76c42..5404d932 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala @@ -1,9 +1,20 @@ /* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ + * This file is part of CoAnSys project. + * Copyright (c) 2012-2017 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ package pl.edu.icm.coansys.document.deduplication import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper @@ -43,6 +54,7 @@ class CartesianTaskSplit( } object CartesianTaskSplit { + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) /** * Combine clusters which have non-empty intersection, so result will be * only separate lists. @@ -70,18 +82,18 @@ object CartesianTaskSplit { /** Split one large cluster into parallel tasks of the given size. */ def parallelizeCluster(clusterId: String, documents: Iterable[DocumentWrapper], tileSize: Int): Seq[CartesianTaskSplit] = { - println(f"Document count: ${documents.size}, tile size $tileSize") + log.info(f"Document count: ${documents.size}, tile size $tileSize") val ntiles = documents.size/tileSize + (if(documents.size % tileSize>0) 1 else 0) println(f"ntiles: $ntiles") - val sdoc = documents.toSeq.sorted(Ordering.by[DocumentWrapper, String](_.getDocumentMetadata.getKey)) - val groupedDocs = sdoc.zipWithIndex.map(docidx => (docidx._2%ntiles, docidx._1)).groupBy[Int](_._1).mapValues(_.map(_._2)) + val sdoc = documents.toVector.sorted(Ordering.by[DocumentWrapper, String](_.getDocumentMetadata.getKey)) + val groupedDocs = sdoc.zipWithIndex.map(docidx => (docidx._2%ntiles, docidx._1)).groupBy[Int](_._1).mapValues(_.map(_._2).toVector).toVector val res = groupedDocs.flatMap(kv => groupedDocs.map(kvin => new CartesianTaskSplit( clusterId, f"${clusterId}_${kv._1}:${kv._2}",kv._2, kvin._2 ) ) ) - res.toSeq + res } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 378c8e1f..0e4c59e7 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -1,3 +1,21 @@ +/* + * This file is part of CoAnSys project. + * Copyright (c) 2012-2017 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ + package pl.edu.icm.coansys.document.deduplication import scala.collection.JavaConversions._ import org.apache.spark.SparkContext @@ -18,13 +36,15 @@ import pl.edu.icm.coansys.document.deduplication.merge.DuplicatesMerger import pl.edu.icm.coansys.models.DocumentProtos import pl.edu.icm.coansys.models.DocumentProtos._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer import pl.edu.icm.coansys.document.deduplication._ import scala.collection.JavaConverters._ +/** Main application for the deduplication of the documents. + * + */ object DeduplicateDocuments { val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) @@ -47,6 +67,7 @@ object DeduplicateDocuments { } } + def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Seq[String] = { val keySizes = initialClusteringKeySize to maximumClusteringKeySize var res = MultiLengthTitleKeyGenerator.generateKeys(doc)(keySizes) @@ -58,7 +79,9 @@ object DeduplicateDocuments { /** * Group items into large clusters, within which detailed analysis will be - * held. Items are grouped by keys generated from the normalised titles. + * held. + * + * Items are grouped by keys generated from the normalised titles. * If the cluster is too big, then longer keys are used, so smaller clusters are * generated. Treshold is maximumClusterSize. * @@ -73,7 +96,7 @@ object DeduplicateDocuments { doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)) //we loose documents here, ony ids are preseved val clusterDoc = idClusterKeys.flatMap(kv => kv._2.map(idcluster => (idcluster, kv._1))) // (clusterId => docId) val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) //(clusterId => clusterSize) - + //build rdd (docId, (clusterId, clusterSize) ) val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) //build rdd - (docId, clusterId) @@ -101,12 +124,21 @@ object DeduplicateDocuments { res } + /** + * Merge the documents using appropriate document merger. + */ def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { val merger = buildDocumentsMerger() val merged = merger.merge(docs); merged } + /** + * Defines comparator according to the weights resulting from experiments. + * + * This is reimplementation of the original Spring XML bean definition, which + * was unnecessary complication at this moment. + */ def buildWorkComparator(): WorkComparator = { val result = new VotesProductComparator; result.setMinVotersWeightRequired(1.5f) @@ -165,18 +197,60 @@ object DeduplicateDocuments { result; } + case class Config( inputFile: String = "", outputFile: String = "", dumpClusters: Boolean = false, keySizeMin: Int = 5, - keySizeMax: Int = 10, + keySizeMax: Int = 15, clusterSizeMax: Int = 500, - tileSize: Int = 25 + tileSize: Int = 25, + filterInvalidDocuments: Boolean = false, + removeDuplicateDocuments: Boolean = false ) - /** - * @param args the command line arguments + + /** Load the documents from the given sequence file, do the optional + * cleanups. + * */ + def loadDocuments( sc: SparkContext, file: String, + filterInvalid: Boolean, removeDoubles: Boolean):RDD[(String, DocumentWrapper)] = { + val rawbytes = sc.sequenceFile[String, BytesWritable](file).mapValues(_.copyBytes) + println("Loaded raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if (filterInvalid) { + val x = dirtyWrappers.filter(w => isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") + x + } else { + dirtyWrappers + } + + if (removeDoubles) { + fixedWrappers.reduceByKey((x, y) => y) + } else { + fixedWrappers + } + } + + /** Debug method to printout top clusters. */ + def printTopClusters(finalClusters:RDD[(String, Seq[String])], count:Int):Unit = { + val finclSizes = finalClusters.mapValues(_.size).takeOrdered(100)(Ordering[Int].on(-_._2)) + println("Top 100 cluster sizes:") + finclSizes.foreach(println(_)) + println("-----\n\n") + + } + + + + def main(args: Array[String]): Unit = { val fixInvalidDocuments = true; val removeDoubles = true; @@ -185,10 +259,13 @@ object DeduplicateDocuments { val parser = new scopt.OptionParser[Config]("CoAnSys Deduplicate Documents") { head("Deduplicate documents", "0.1") - -// opt[Unit]('c', "dump-clusters").action((x, c) => -// c.copy(dumpClusters = true)).text("dump similarity clusters into debug/clusters hdfs dir during run") - + + opt[Unit]('f', "filter-invalid").action((x, c) => + c.copy(filterInvalidDocuments = true)).text("filter invalid (empty) documents before run.") + + opt[Unit]('d', "remove-doubles").action((x, c) => + c.copy(removeDuplicateDocuments = true)).text("filter out duplicates sharing the same key before processing.") + opt[Int]("cluster-key-min").abbr("kmn").action((x, c) => c.copy(keySizeMin = x)). validate(x => if (x >= 2) success @@ -222,12 +299,13 @@ object DeduplicateDocuments { config case None => // arguments are bad, error message will have been displayed - println("No config.") + println("No config, aborting.") return } println("Creating context...") + //required to operate protobuf correctly val conf = new SparkConf() .setAppName("Document deduplication") .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") @@ -242,52 +320,27 @@ object DeduplicateDocuments { // val inputDocuments = "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" // val inputDocuments = "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" val outputDocuments = cfg.outputFile - val tileSize = cfg.tileSize - val initialClusteringKeySize = cfg.keySizeMin - val maximumClusteringKeySize = cfg.keySizeMax - val maximumClusterSize = cfg.clusterSizeMax - - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - println("Loaded raw bytes.") - - val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) - - //fix invalid documents: - val fixedWrappers = if (fixInvalidDocuments) { - val x = dirtyWrappers.filter(w => isValidDocument(w._2)) - val afterSize = x.count; - val preSize = dirtyWrappers.count - log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") - x - } else { - dirtyWrappers - } - - val wrappers = if (removeDoubles) { - fixedWrappers.reduceByKey((x, y) => y) - } else { - fixedWrappers - } - + + + //load documents + val wrappers = loadDocuments(sc, inputDocuments, cfg.filterInvalidDocuments, cfg.removeDuplicateDocuments) val initialSize = wrappers.count println(f"Starting processing with $initialSize documents.") - val initialGroups = prepareInitialClustering(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) - initialGroups.persist //(StorageLevel.MEMORY_AND_DISK) + val initialGroups = prepareInitialClustering(wrappers, cfg.keySizeMin, cfg.keySizeMax, cfg.clusterSizeMax) + initialGroups.persist val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) val initialClusterCount = clustersToDeduplicate.count //TODO: some statistics here on cluster, would be useful. -// val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) - val tiledTasks = clustersToDeduplicate.flatMap(p => CartesianTaskSplit.parallelizeCluster(p._1, p._2, tileSize)) + val tiledTasks = clustersToDeduplicate.flatMap(p => CartesianTaskSplit.parallelizeCluster(p._1, p._2, cfg.tileSize)) - tiledTasks.persist //(StorageLevel.MEMORY_AND_DISK) + tiledTasks.persist val tileCount = tiledTasks.count; println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") - // val comparator = sc.broadcast(buildWorkComparator) //todo: can we do comparator broadcast variable? - + //build (clusterId, Seq(docId)) rdd: val partialEqualityClusters = tiledTasks.flatMap( task => { @@ -319,16 +372,15 @@ object DeduplicateDocuments { cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) } ) + //now we got all the items in place - finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); - val finclSizes = finalClusters.mapValues(_.size).takeOrdered(100)(Ordering[Int].on(-_._2)) - println("Top 100 cluster sizes:") - finclSizes.foreach(println(_)) - println("-----\n\n") + finalClusters.persist + + printTopClusters(finalClusters, 100) //count clusters, documents in clusters and number of documents to be deduced: val finalClusterCount = finalClusters.count - val documentInFinalClusterCount = finalClusters.map(_._2.size).reduce(_ + _) + val documentInFinalClusterCount = finalClusters.map(_._2.size).fold(0)(_ + _) val documentRemovedDuringClusteringCount = documentInFinalClusterCount - finalClusterCount println(f"Finally created $finalClusterCount clusters, containing $documentInFinalClusterCount documents, $documentRemovedDuringClusteringCount documents will be removed.") @@ -337,7 +389,7 @@ object DeduplicateDocuments { map(v => (v._2, v._1)) val documentWrappersPrepared = wrappers.leftOuterJoin(docIdWithClusterId); val mergedDocuments = documentWrappersPrepared.filter(_._2._2.isDefined). - map(x => (x._2._2, List(x._2._1))).reduceByKey(_ ++ _). //get lists of cluster documents by cluster id + map(x => (x._2._2, List(x._2._1))).foldByKey(List())(_ ++ _). //get lists of cluster documents by cluster id map(kv => { val doc = mergeDocuments(kv._2) (doc.getDocumentMetadata.getKey, doc) diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala index 24ec2cc2..c1ae7b8e 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala @@ -1,6 +1,6 @@ /* * This file is part of CoAnSys project. - * Copyright (c) 2012-2015 ICM-UW + * Copyright (c) 2012-2017 ICM-UW * * CoAnSys is free software: you can redistribute it and/or modify * it under the terms of the GNU Affero General Public License as published by diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala new file mode 100644 index 00000000..39b358c0 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala @@ -0,0 +1,124 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import com.holdenkarau.spark.testing.SharedSparkContext +import org.scalatest.FunSuite +import org.scalatest.GivenWhenThen +import pl.edu.icm.coansys.models.DocumentProtos +import pl.edu.icm.coansys.models.DocumentProtos._ + +class DeduplicateDocumentTest extends FunSuite with GivenWhenThen with SharedSparkContext { + + test("docaument validation") { + Given("Empty document") + val doc = DocumentWrapper.newBuilder().setRowId("test").build; + When("We validate") + Then("Document is invalid") + assert(!DeduplicateDocuments.isValidDocument(doc)) + Given("Doc with empty metadata") + val doc2 = DocumentWrapper.newBuilder(doc).setDocumentMetadata( + DocumentMetadata.newBuilder + .setBasicMetadata(BasicMetadata.newBuilder.build) + .setKey("Key") + .build + ).build + When("We test if it is valid") + + Then("It is not valid") + assert(!DeduplicateDocuments.isValidDocument(doc2)) + + Given("Doc with title ") + val doc3 = DocumentWrapper.newBuilder(doc2).setDocumentMetadata( + DocumentMetadata.newBuilder() + .setBasicMetadata(BasicMetadata.newBuilder().addTitle(TextWithLanguage.newBuilder.setText("Title")).build) + .setKey("key") + .build + ) + .build + When("We test if it is valid: ") + assert(DeduplicateDocuments.isValidDocument(doc3)) + Then("It is not valid") + } + + test("Sample with spark context") { + Given("RDD of sequence 1 to n (n=100)") + val n = 100 + val rdd = sc.parallelize(1 to n) + When("We sum") + val sum = rdd.sum + Then("result is n*(n+1)/2") + assertResult(n * (n + 1) / 2)(sum) + + } + + + def createDocument(key:String , title:String):DocumentWrapper = { + DocumentProtos.DocumentWrapper.newBuilder().setDocumentMetadata( + DocumentProtos.DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( + DocumentProtos.BasicMetadata.newBuilder().addTitle( + DocumentProtos.TextWithLanguage.newBuilder().setText(title))) + ).setRowId(key).build(); + } + + + test("Initial clustering test:") { + Given("Data set has the same title begninnings") + + val d3 = (1 to 10).map(x=> createDocument("id_aaa"+x, "aaa")) + val d4 = (1 to 10).map(x=> createDocument("id_aaaa"+x, "aaaa")) + val d5 = (1 to 10).map(x=> createDocument("id_aaaaa"+x, "aaaaa")) + val d12 = (1 to 10).map(x=> createDocument("id_aaaaaaaaaaa"+x, "aaaaaaaaaa"+x)) + val docs = List()++d3++d4++d5++d12; + val input = sc.parallelize(docs).map(doc=> (doc.getRowId, doc)) + When("We build clustering with short key") + val r1 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 2, 20) + Then("We get only one cluster, with all documents:") + val r1c = r1.collect + assert(r1c.size==1) + assert(r1c(0)._2.size==40) + And("Key is 1st and 3rd letter") + assert(r1c(0)._1=="aa") + When("We build clustering with variable key 2-3") + val r2 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 3, 10) + Then("We get only two clusters:") + val r2c = r2.collect + assert(r2c.size==2) + val r2cm = r2c.toMap + assert(r2cm("aa").size==20) + assert(r2cm("aaa").size==20) +// r2c.flatMap(_._2).map(_.getKey()) +// + When("We build clustering with variable key 2-5") + val r3 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 5, 10) + Then("We get 3 clusters:") + val r3c = r3.collect + assert(r3c.size==3) + val r3cm = r3c.toMap + assert(r3cm("aa").size==20) + assert(r3cm("aaa").size==10) + assert(r3cm("aaaa").size==10) + + When("We build clustering with variable key 2-6") + val r4 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 6, 9) + Then("We get 11 clusters:") + val r4c = r4.collect + assert(r4c.size==11) + val r4cm = r4c.toMap + assert(r4cm("aa").size==20) + assert(r4cm("aaa").size==10) + assert(r4cm("aaaaa2").size==1) + assert(r4cm("aaaaa1").size==2) + + + } + +} From 1a2c5dd14ed6da1a6c8a4101df3785131237d3da Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Mon, 24 Jul 2017 11:49:28 +0200 Subject: [PATCH 10/20] Fixed oozie workflow building. --- .../pom.xml | 10 +- .../src/main/oozie/workflow.xml | 101 ++++++++++++++++++ 2 files changed, 109 insertions(+), 2 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml diff --git a/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml index e1a28320..1c5222a6 100644 --- a/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml @@ -1,6 +1,6 @@ - deduplication-document + deduplication-document-spark pl.edu.icm.coansys 1.11-SNAPSHOT @@ -11,6 +11,12 @@ UTF-8 + + + GNU AFFERO GENERAL PUBLIC LICENSE, Version 3 (AGPL-3.0) + http://opensource.org/licenses/AGPL-3.0 + + @@ -23,7 +29,7 @@ pl.edu.icm.coansys - deduplication-document-impl + deduplication-document-spark-impl ${project.version} diff --git a/deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml b/deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml new file mode 100644 index 00000000..02d2053a --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml @@ -0,0 +1,101 @@ + + + + + jobTracker + + + nameNode + + + queueName + default + + + input + ${inputSeqFile} + + + output + ${outputSeqFile} + + + + sparkExecutorMemory + 128G + memory for individual executor + + + sparkExecutorCores + 16 + number of cores used by single executor + + + sparkExecutorsNumber + 16 + total number of executors + + + sparkHistoryServer + http://spark-m2.vls.icm.edu.pl:18080 + Address of spark history server + + + sparkEventLogDir + hdfs:/user/spark/applicationHistory + Directory for spark events logging + + + + + + + + + + ${jobTracker} + ${nameNode} + + + + + + yarn-cluster + cluster + document-deduplication-spark + + pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + + ${comacWfPath}/lib/citation-matching-coansys-code-${project.version}.jar + + --conf spark.network.timeout=10000000 --conf spark.executor.heartbeatInterval=10000000 --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --num-executors ${sparkExecutorsNumber} --conf spark.yarn.historyServer.address=${sparkHistoryServer} --conf spark.eventLog.dir=${sparkEventLogDir} --conf spark.eventLog.enabled=true + f + d + -ts + 50 + ${input} + ${output} + + + + + + + + Workflow failed, error message [${wf:errorMessage(wf:lastErrorNode())}] + + + From ca4592bb832baf6a267872d1124fcefdc62c5a5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Mon, 24 Jul 2017 13:22:35 +0200 Subject: [PATCH 11/20] Cleaning up project files --- .gitignore | 2 ++ .../deduplication-document-impl/pom.xml | 17 +++++++++++++++++ pom.xml | 1 + 3 files changed, 20 insertions(+) diff --git a/.gitignore b/.gitignore index ad354819..7dc60f54 100644 --- a/.gitignore +++ b/.gitignore @@ -10,6 +10,7 @@ # Netbeans files # nb-configuration.xml +nbaction.xml # IntelliJ IDEA files # .idea @@ -43,3 +44,4 @@ dependency-reduced-pom.xml /affiliation-organization-matching/affiliation-organization-matching-workflow/src/main/oozie/workflow.xml /deduplication-organization/deduplication-organization-workflow/src/main/oozie/workflow.xml +/deduplication-document-spark/deduplication-document-spark-impl/nbproject/ diff --git a/deduplication-document/deduplication-document-impl/pom.xml b/deduplication-document/deduplication-document-impl/pom.xml index b3b13ce5..4fb1514c 100644 --- a/deduplication-document/deduplication-document-impl/pom.xml +++ b/deduplication-document/deduplication-document-impl/pom.xml @@ -35,6 +35,23 @@ 1.8 + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + + + package + + shade + + + + + diff --git a/pom.xml b/pom.xml index 2548a7ed..d17b9283 100644 --- a/pom.xml +++ b/pom.xml @@ -198,6 +198,7 @@ citation-matching commons deduplication-document + deduplication-document-spark deduplication-organization disambiguation disambiguation-author From ece39dcf5e95f22460e3bf90339e1e8f62d1ca56 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 17 Mar 2017 16:32:26 +0100 Subject: [PATCH 12/20] Initial version of spark-based document deduplication. It contains a new version of the clustering mechanism with auto-sizing clusters. --- .../nbactions.xml | 73 ++++ .../deduplication-document-spark-impl/pom.xml | 161 ++++++++ .../CustomOddsCharsKeyGenerator.java | 136 +++++++ .../deduplication/DeduplicateDocuments.scala | 298 +++++++++++++++ .../DocumentWrapperKryoRegistrator.java | 80 ++++ .../merge/AdvancedDuplicatesMerger.java | 349 ++++++++++++++++++ .../deduplication/merge/DuplicatesMerger.java | 13 + .../merge/SimpleDuplicatesMerger.java | 98 +++++ .../CustomOddsCharsKeyGeneratorTest.java | 110 ++++++ .../pom.xml | 35 ++ deduplication-document-spark/pom.xml | 20 + 11 files changed, 1373 insertions(+) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/pom.xml create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java create mode 100644 deduplication-document-spark/deduplication-document-spark-workflow/pom.xml create mode 100644 deduplication-document-spark/pom.xml diff --git a/deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml b/deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml new file mode 100644 index 00000000..b9137ba9 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/nbactions.xml @@ -0,0 +1,73 @@ + + + + run + + jar + + + process-classes + org.codehaus.mojo:exec-maven-plugin:1.2.1:exec + + + -classpath %classpath pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + java + + + + debug + + jar + + + process-classes + org.codehaus.mojo:exec-maven-plugin:1.2.1:exec + + + -Xdebug -Xrunjdwp:transport=dt_socket,server=n,address=${jpda.address} -classpath %classpath pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + java + true + + + + profile + + jar + + + process-classes + org.codehaus.mojo:exec-maven-plugin:1.2.1:exec + + + -classpath %classpath pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + java + + + + CUSTOM-scala:run + scala:run + + scala:run + + + + CUSTOM-RunSmall + RunSmall + + scala:run + + + test|test2 + + + + + CUSTOM-clean,build,upload + clean,build,upload + + clean + install + wagon:upload-single + + + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml new file mode 100644 index 00000000..6461abb8 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -0,0 +1,161 @@ + + + 4.0.0 + + pl.edu.icm.coansys + deduplication-document-spark + 1.11-SNAPSHOT + + + deduplication-document-spark-impl + jar + Deduplication - Document - SparkVersion - Implementation + + + + ssh-cypisek + scpexe://cypisek/jobs + + + + + src/main/scala + src/test/scala + + + + + net.alchim31.maven + scala-maven-plugin + + + + + compile + testCompile + + + + + -dependencyfile + ${project.build.directory}/.scala_dependencies + + + + + + + + base + + pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-shade-plugin + 2.3 + + + package + + shade + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + junit:junit + log4j:log4j:jar: + org.scala-lang:scala-library:jar: + org.apache.spark:spark-core_2.10 + org.apache.spark:spark-sql_2.10 + org.apache.spark:spark-streaming_2.10 + + + ${project.artifactId}-${project.version} + + + + org.codehaus.mojo + wagon-maven-plugin + 1.0-beta-3 + + ${project.build.directory}/${project.build.finalName}.jar + scp://cypisek-gw.ocean.icm.edu.pl/home/axnow/jobs/ + dedupdocs.jar + + + + + + + + + + org.apache.maven.wagon + wagon-ssh + 2.8 + + + + + + + + ${project.groupId} + models + ${project.version} + + + ${project.groupId} + deduplication-document-impl + ${project.version} + + + org.apache.spark + spark-core_2.10 + + + org.apache.spark + spark-graphx_2.10 + + + javax.servlet + javax.servlet-api + 3.1.0 + runtime + + + com.google.guava + guava + 15.0 + + + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java new file mode 100644 index 00000000..442208ff --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java @@ -0,0 +1,136 @@ +/* + * This file is part of CoAnSys project. + * Copyright (c) 2012-2015 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.Arrays; +import pl.edu.icm.coansys.commons.java.DocumentWrapperUtils; +import pl.edu.icm.coansys.commons.java.StringTools; +import pl.edu.icm.coansys.deduplication.document.keygenerator.WorkKeyGenerator; +import pl.edu.icm.coansys.models.DocumentProtos; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; + +/** + * An early deduplication phase key used to group works into clusters. This is + * an extended version, which allows to define requested key size + * + * @author Łukasz Dumiszewski + * @author Aleksander Nowiński + * + */ +public class CustomOddsCharsKeyGenerator implements WorkKeyGenerator { + + public static final int DEFAULT_KEY_SIZE = 5; + + private int keySize = DEFAULT_KEY_SIZE; + + private int[] keySizes = {DEFAULT_KEY_SIZE}; + + public CustomOddsCharsKeyGenerator() { + } + + public CustomOddsCharsKeyGenerator(int keySize) { + this.keySize = keySize; + } + + /** + * Generates reasonable deduplication key for the given + * {@link DocumentWrapper}. The key has size as defined by keySize param. + * The key is created using title of the work, with removed punctuation and + * basic english stopwords, and then keySize odd letters are taken. The + * method is thread-safe. + * + * @param doc document, which key is generated for + * @return an reasonable key for starting deduplication of size keySize + */ + @Override + public String generateKey(DocumentProtos.DocumentMetadata doc) { + String title = DocumentWrapperUtils.getMainTitle(doc); + return generateKey(title); + } + + /** + * Generate a collection of keys of predefined sizes. + * + * @param doc + * @return + */ + public String[] generateKeyList(DocumentProtos.DocumentMetadata doc) { + String title = DocumentWrapperUtils.getMainTitle(doc); + return generateKeys(title); + } + + protected String[] generateKeys(String title) { + title = cleanUpString(title); + String[] res = new String[keySizes.length]; + for (int k = 0; k < keySizes.length; k++) { + int kl = keySizes[k]; + StringBuilder oddCharsSB = new StringBuilder(); + for (int i = 0; i < title.length() && oddCharsSB.length() < kl; i += 2) { + oddCharsSB.append(title.charAt(i)); + } + res[k] = oddCharsSB.toString(); + } + return res; + } + + protected String generateKey(String title) { + title = cleanUpString(title); + + StringBuilder oddCharsSB = new StringBuilder(); + for (int i = 0; i < title.length() && oddCharsSB.length() < keySize; i += 2) { + oddCharsSB.append(title.charAt(i)); + } + return oddCharsSB.toString(); + } + + protected String cleanUpString(String title) { + title = StringTools.normalize(title); //fixme: it seems that normalize, despite javadocs has stopword removal already + title = StringTools.removeStopWords(title); + title = title.replaceAll("\\s", ""); + return title; + } + + public int getKeySize() { + return keySize; + } + + public void setKeySize(int keySize) { + this.keySize = keySize; + } + + public int[] getKeySizes() { + return Arrays.copyOf(keySizes, keySizes.length); + } + + public void setKeySizes(int[] keySizes) { + if (keySizes == null) { + throw new IllegalArgumentException("Null sizes not premitted"); + } + if (keySizes.length < 1) { + throw new IllegalArgumentException("Non empty array required"); + } + for (int i = 0; i < keySizes.length - 1; i++) { + if (keySizes[i] >= keySizes[i + 1]) { + throw new IllegalArgumentException("Array must be sorted in growing order and no equal sizes present."); + } + + } + this.keySizes = Arrays.copyOf(keySizes, keySizes.length); + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala new file mode 100644 index 00000000..72e33920 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -0,0 +1,298 @@ +package pl.edu.icm.coansys.document.deduplication +import scala.collection.JavaConversions._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.hadoop.io.BytesWritable +import org.apache.log4j.Level +import org.apache.log4j.Logger +import org.apache.spark.SparkConf +import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter +import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter +import pl.edu.icm.coansys.deduplication.document.voter.IssueVolumeVoter +import pl.edu.icm.coansys.deduplication.document.voter.JournalVoter +import pl.edu.icm.coansys.deduplication.document.voter.PagesVoter +import pl.edu.icm.coansys.deduplication.document.voter.SimilarityVoter +import pl.edu.icm.coansys.deduplication.document.voter.TitleVoter +import pl.edu.icm.coansys.deduplication.document.voter.YearVoter +import pl.edu.icm.coansys.document.deduplication.merge.AdvancedDuplicatesMerger +import pl.edu.icm.coansys.document.deduplication.merge.DuplicatesMerger +import pl.edu.icm.coansys.models.DocumentProtos +import pl.edu.icm.coansys.models.DocumentProtos._ +import org.apache.spark.rdd.RDD +import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator +import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator +import scala.collection.mutable.ListBuffer + +object DeduplicateDocuments { + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) + val initialClusteringKeySize = 5 + val maximumClusteringKeySize = 15 + val maximumClusterSize = 300 + //max author count - ignored + + // def calculateKey(doc: DocumentMetadata): String = { + // new OddsCharsKeyGenerator().generateKey(doc) + // } + + def calculateKey(doc: DocumentMetadata, size: Int): String = { + new CustomOddsCharsKeyGenerator(size).generateKey(doc) + } + + def prepareClusters(inputDocs: RDD[DocumentWrapper]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Started preparation of clusters.") + var keySize = initialClusteringKeySize + + var approvedClusters: RDD[(String, Iterable[DocumentWrapper])] = null + var docs = inputDocs + var docCount = 0L + var iteration = 1 + + while ({ docCount = docs.count; docCount } > 0 && keySize < maximumClusteringKeySize) { + log.info("Starting iteration %d, keySize: %d, docs to count: %d".format(iteration, keySize, docCount)) + + var processedClusters = docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey() + var posClusters = processedClusters.filter(p => p._2.size <= maximumClusterSize) + if (approvedClusters == null) { + approvedClusters = posClusters + } else { + approvedClusters = approvedClusters.union(posClusters) + } + docs = processedClusters.filter(p => p._2.size > maximumClusterSize).flatMap(p => p._2) + keySize += 1 + iteration += 1 + } + log.info("Finished loop, keySize: %d, iterations done: %d, docs left: %d".format(keySize - 1, iteration - 1, docCount)) + if (docCount > 0) { + log.info("Adding leftovers (%d documents) with keySize %d".format(docCount, keySize)) + approvedClusters = approvedClusters.union(docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey()) + } + approvedClusters + } + + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supported, exitting.") + return ; + } else { + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) + val outputDocuments = args(1) + + // val debugDir = if (args.size > 2) Some(args(2)) else None + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + log.info("Loaded raw bytes.") + // rawbytes.count() + // log.info("Counted raw bytes.") + + val wrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + +// val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() + val initialGroups = prepareClusters(wrappers.map(_._2)) + log.info("After initial group preparation count.") + // val igs = initialGroups.count() + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + // val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) + val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) + val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) + val timing = timedDeduplicated.map(x => x._2) + + log.info("After reducing clusters (comparison)") + val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) + log.info("Finished merge") + // val mergedSize = merged.count() + + val single = initialGroups.filter(t => t._2.size == 1) + // log.info("Got initial group count=" + igs + "; singular=" + single.count + + // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) + // + //now merge the arrays: + val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) + if ("-" != outputDocuments) { + val bas = toWrite.mapValues(doc => doc.toByteArray()) + bas.saveAsSequenceFile(outputDocuments); + } + val tgrouped = timing.groupByKey; + val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect + println("================ Timing stats ======================") + stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) + println("================ end of timing stats ======================") + + println("Exit") + + } + + def buildDocumentsMerger(): DuplicatesMerger = { + val res = new AdvancedDuplicatesMerger + res.setup("") + res + } + + def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { + val merger = buildDocumentsMerger() + val merged = merger.merge(docs); + merged + } + + def buildWorkComparator(): WorkComparator = { + val result = new VotesProductComparator; + result.setMinVotersWeightRequired(1.5f) + result.setProbabilityTreshold(0.5f) + result.setTresholdIncreasingVotersRequired(0.7f) + + val voters = new ListBuffer[SimilarityVoter]() + val dv = new DoiVoter() + dv.setWeight(1.0f) + voters += dv + val jv = new JournalVoter() + jv.setWeight(0.3f) + jv.setDisapproveLevel(0.5f) + jv.setApproveLevel(0.05f) + voters += jv + + val wivv = new IssueVolumeVoter + wivv.setWeight(0.3f) + wivv.setAbstainIfAbsent(true) + wivv.setSubsetResult(0.8f) + wivv.setPartiallyMatchResult(0.52f) + voters += wivv + + val wpv = new PagesVoter + wpv.setWeight(.3f) + wpv.setAbstainIfAbsent(true) + wpv.setAbsentResult(0.6f) + wpv.setSubsetResult(0.75f) + wpv.setPartiallyMatchResult(0.64f) + wpv.setRemoveRepeated(true) + voters += wpv + + val wyv = new YearVoter + wyv.setWeight(.3f) + wyv.setAbstainIfAbsent(true) + wyv.setAbsentResult(.52f) + wyv.setSubsetResult(.9f) + wyv.setPartiallyMatchResult(.75f) + wyv.setRemoveRepeated(true) + voters += wyv + + val wtv = new TitleVoter() + wtv.setWeight(0.8f) + wtv.setDisapproveLevel(0.11f) + wtv.setApproveLevel(0.001f) + wtv.setMaxNormalizedTitleLength(90) + voters += wtv + + val wav = new AuthorsVoter + wav.setWeight(0.8f) + wav.setDisapproveLevel(0.2f) + wav.setApproveLevel(0.03f) + voters += wav + + result.setSimilarityVoters(voters) + result; + } + + //size, num, min time, max time, avg time, avg_group_count + def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { + val count = measurements.size + val times = measurements.map(_._2) + val minTime = times.min / 1000.0 + val maxTime = times.max / 1000.0 + val avgTime = times.sum / (1000.0 * count) + val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) + (size, count, minTime, maxTime, avgTime, avgGroupSize); + } + + def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { + log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) + + val res = Array.ofDim[Int](cluster.size, cluster.size) + val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) + val comparator = buildWorkComparator() + + val sind = sorted.toList.zipWithIndex + + val procStart = System.currentTimeMillis + sind.foreach(p1 => { + val i1 = p1._2 + val d1 = p1._1 + //make sure diagonal is zeroed + (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) + //simple iteration over table + sind.foreach(p2 => { + val i2 = p2._2 + val d2 = p2._1 + if (i1 < i2) { + val s = System.currentTimeMillis + comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) + val e = System.currentTimeMillis + res(i1)(i2) = (e - s).toInt + } + }) + }) + val procEnd = System.currentTimeMillis + val elapsedSteps = res.map(_.sum).sum + log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) + + (sorted, res) + } + + //todo: timing depending on size + //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which + //are assumed to be duplicates. + def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { + log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) + // + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + if (cluster.size > 30) { + + log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) + } + val startTime = System.currentTimeMillis(); + val classes = ListBuffer[ListBuffer[DocumentWrapper]]() + val comparator = buildWorkComparator() + cluster.foreach(x => { + val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) + if (arr.isEmpty) { + val nclass = ListBuffer(x) + classes += nclass + } else { + arr.get += x + } + }) + val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 + val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) + log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") + + res + + } + + def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { + val start = System.currentTimeMillis; + val rresult = deduplicateCluster(cluster, clusterId); + val end = System.currentTimeMillis + + (rresult, (cluster.size, (rresult.size, end - start))) + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java new file mode 100644 index 00000000..726a0bf1 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java @@ -0,0 +1,80 @@ +package pl.edu.icm.coansys.document.deduplication; + + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.spark.serializer.KryoRegistrator; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; + +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +/** + * + * @author Aleksander Nowinski + */ +public class DocumentWrapperKryoRegistrator implements KryoRegistrator { + + @Override + public void registerClasses(Kryo kryo) { + kryo.register(DocumentWrapper.class, new DocumentWrapperSerializer()); + kryo.register(DocumentMetadata.class, new DocumentMetadataSerializer()); + } + + + public static class DocumentWrapperSerializer extends Serializer { + + @Override + public void write(Kryo kryo, Output output, DocumentWrapper object) { + byte[] bytes = object.toByteArray(); + + output.writeInt(bytes.length, true); + output.writeBytes(bytes); + } + + @Override + public DocumentWrapper read(Kryo kryo, Input input, Class type) { + int length = input.readInt(true); + byte[] bytes = input.readBytes(length); + + try { + return DocumentWrapper.parseFrom(bytes);//FIXME: is this exception handling ok? + } catch (InvalidProtocolBufferException ex) { + throw new RuntimeException(ex); + } + } + + } + + public static class DocumentMetadataSerializer extends Serializer { + + @Override + public void write(Kryo kryo, Output output, DocumentMetadata object) { + byte[] bytes = object.toByteArray(); + + output.writeInt(bytes.length, true); + output.writeBytes(bytes); + } + + @Override + public DocumentMetadata read(Kryo kryo, Input input, Class type) { + int length = input.readInt(true); + byte[] bytes = input.readBytes(length); + + try { + return DocumentMetadata.parseFrom(bytes);//FIXME: is this exception handling ok? + } catch (InvalidProtocolBufferException ex) { + throw new RuntimeException(ex); + } + } + + } + +} \ No newline at end of file diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java new file mode 100644 index 00000000..af45ee69 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/AdvancedDuplicatesMerger.java @@ -0,0 +1,349 @@ +package pl.edu.icm.coansys.document.deduplication.merge; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.UUID; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.commons.lang.StringUtils; + +import pl.edu.icm.coansys.commons.java.DiacriticsRemover; +import pl.edu.icm.coansys.commons.java.Pair; +import pl.edu.icm.coansys.models.DocumentProtos.Author; +import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import pl.edu.icm.coansys.models.DocumentProtos.KeyValue; +import pl.edu.icm.coansys.models.DocumentProtos.KeywordsList; + +/** + * Class which merge list of DocumentWrapper to one single DocumentWrapper + * object + * + * @author acz + */ +public class AdvancedDuplicatesMerger implements DuplicatesMerger { + public static final String MERGED_ID_SEPARATOR = "+"; + private Map collectionPreferences; + + @Override + public void setup(String collectionPriorities) { + collectionPreferences = new HashMap(); + + for (String coll : collectionPriorities.split("," )) { + coll = coll.trim(); + + Pattern prefPattern = Pattern.compile("^(-?\\d+):(.+)"); + Matcher prefMatcher = prefPattern.matcher(coll); + if (prefMatcher.matches()) { + String priority = prefMatcher.group(1); + String collectionName = prefMatcher.group(2); + collectionPreferences.put(collectionName, Integer.parseInt(priority)); + + } + } + } + + /** + * Chooses the best DocumentWrapper, updates keys in DocumentWrapper, + * DocumentMetadata and authors, gets extIds, auxiliarInfos from all + * DocumentWrappers, matches authors, gets extIds from matched authors + * + * @param duplicates + * @return + */ + @Override + public DocumentWrapper merge(List duplicates) { + + if (duplicates == null || duplicates.isEmpty()) { + throw new RuntimeException("Nothing to merge"); + } else if (duplicates.size() == 1) { + return duplicates.get(0); + } + + int theBestDocumentWrapperIndex = chooseTheBestIndex(duplicates); + + // Collect information from all items + List identifiers = new ArrayList(duplicates.size()); + List allExtIds = new ArrayList(); + List allAuxiliarInfos = new ArrayList(); + SortedSet sortedCollections = new TreeSet(); + List allKeywords = new ArrayList(); + + for (DocumentWrapper dw : duplicates) { + DocumentMetadata dm = dw.getDocumentMetadata(); + identifiers.add(dw.getRowId()); + List collectionList = new ArrayList(dm.getCollectionList()); + if (collectionList.isEmpty()) { + collectionList.add("unknown"); //TODO move to constants + } + for (String collection : collectionList) { + sortedCollections.add(collection); + } + allExtIds.addAll(dm.getExtIdList()); + allAuxiliarInfos.addAll(dm.getAuxiliarInfoList()); + allKeywords.addAll(dm.getKeywordsList()); + } + Collections.sort(identifiers); + String joinedIds = StringUtils.join(identifiers, MERGED_ID_SEPARATOR); + String newIdentifier = UUID.nameUUIDFromBytes(joinedIds.getBytes()).toString(); + + // Create new DocumentWrapper.Builder + DocumentWrapper.Builder resultBuilder = DocumentWrapper + .newBuilder(duplicates.get(theBestDocumentWrapperIndex)); + + // Modify fields of DocumentWrapper.Builder + resultBuilder.setRowId(newIdentifier); + + DocumentMetadata.Builder documentMetadataBuilder = resultBuilder + .getDocumentMetadataBuilder(); + + BasicMetadata.Builder basicMetadataBuilder = documentMetadataBuilder + .getBasicMetadataBuilder(); + documentMetadataBuilder.setKey(newIdentifier); + documentMetadataBuilder.addAllOrigKey(identifiers); + documentMetadataBuilder.addAllCollection(sortedCollections); + + List finalAuthorBuilderList = basicMetadataBuilder + .getAuthorBuilderList(); + for (Author.Builder authorBuilder : finalAuthorBuilderList) { + String positionSuffix = authorBuilder.getKey().replaceAll(".*(#c\\d+)", "$1"); + authorBuilder.setDocId(newIdentifier); + authorBuilder.setKey(newIdentifier + positionSuffix); + } + + List> authorListsToMerge = new ArrayList>(); + + for (int i = 0; i < duplicates.size(); i++) { + if (i != theBestDocumentWrapperIndex) { + List unmatchedList = duplicates.get(i) + .getDocumentMetadata().getBasicMetadata() + .getAuthorList(); + List matchedList = matchAuthors(finalAuthorBuilderList, + unmatchedList); + if (matchedList != null) { + authorListsToMerge.add(matchedList); + } + } + } + + mergeAuthors(finalAuthorBuilderList, authorListsToMerge); + + documentMetadataBuilder.clearExtId(); + documentMetadataBuilder.addAllExtId(mergeKeyValues(allExtIds)); + documentMetadataBuilder + .addAllAuxiliarInfo(mergeKeyValues(allAuxiliarInfos)); + documentMetadataBuilder.addAllKeywords(mergeKeywords(allKeywords)); + + // Build and return DocumentWrapper + return resultBuilder.build(); + } + + /** + * Moves some informations from author lists in listsToMerge to base list + * + * @param base + * @param listsToMerge + */ + protected void mergeAuthors(List base, + List> listsToMerge) { + + for (int i = 0; i < base.size(); i++) { + Author.Builder baseBuilder = base.get(i); + List allExtIds = new ArrayList(); + allExtIds.addAll(baseBuilder.getExtIdList()); + for (List authorsToMerge : listsToMerge) { + Author author = authorsToMerge.get(i); + if (author != null) { + allExtIds.addAll(author.getExtIdList()); + } + } + + baseBuilder.clearExtId(); + baseBuilder.addAllExtId(mergeKeyValues(allExtIds)); + } + } + + /** + * Checks if tho author lists contain the same authors. Returns second list + * in order as in base list. + * + * @param base + * @param second + * @return + */ + protected List matchAuthors(List base, + List second) { + List result = new ArrayList(base.size()); + List secondCopy = new ArrayList(second); + + for (Author.Builder author : base) { + Author foundAuthor = null; + for (Author secondAuthor : secondCopy) { + + if (equalsIgnoreCaseIgnoreDiacritics( + author.getName(), secondAuthor.getName()) + || equalsIgnoreCaseIgnoreDiacritics( + author.getForenames(), secondAuthor.getForenames()) + && equalsIgnoreCaseIgnoreDiacritics( + author.getSurname(), secondAuthor.getSurname())) { + foundAuthor = secondAuthor; + break; + } + } + if (foundAuthor != null) { + result.add(foundAuthor); + secondCopy.remove(foundAuthor); + } else { + result.add(null); + } + } + + if (result.size() == base.size()) { + return result; + } else { + return null; + } + } + + private boolean equalsIgnoreCaseIgnoreDiacritics(String firstName, + String secondName) { + if (firstName.isEmpty() || secondName.isEmpty()) { + return false; + } + return DiacriticsRemover.removeDiacritics(firstName).equalsIgnoreCase( + DiacriticsRemover.removeDiacritics(secondName)); + } + + /** + * Merges KeyValue messages. Removes repetitions, concatenates comments. + * + * @param listWithRepetitions + * @return + */ + protected static List mergeKeyValues(List listWithRepetitions) { + + Map, String> map = new HashMap, String>(); + for (KeyValue extId : listWithRepetitions) { + Pair keyValue = new Pair( + extId.getKey(), extId.getValue()); + String comment = extId.getComment(); + if (!map.containsKey(keyValue)) { + map.put(keyValue, comment); + } else if (!comment.isEmpty()) { + String oldComment = map.get(keyValue); + if (oldComment.isEmpty()) { + map.put(keyValue, comment); + } else { + map.put(keyValue, oldComment + "\t" + comment); + } + } + } + + List result = new ArrayList(); + + for (Map.Entry, String> mapEntry : map.entrySet()) { + KeyValue.Builder kvBuilder = KeyValue.newBuilder(); + kvBuilder.setKey(mapEntry.getKey().getX()); + kvBuilder.setValue(mapEntry.getKey().getY()); + String comment = mapEntry.getValue(); + if (!comment.isEmpty()) { + kvBuilder.setComment(comment); + } + result.add(kvBuilder.build()); + } + + return result; + } + + /** + * Chooses index of item which will be the base for merged result. + * + * @param duplicates + * @return + */ + protected int chooseTheBestIndex(List duplicates) { + if (collectionPreferences == null || collectionPreferences.isEmpty()) { + return 0; + } + + int bestDuplicateIdx = 0; + int bestPref = Integer.MIN_VALUE; + + for (int i = 0; i < duplicates.size(); i++) { + DocumentWrapper dw = duplicates.get(i); + for (String collection : dw.getDocumentMetadata().getCollectionList()) { + + int pref = 0; + if (collectionPreferences.containsKey(collection)) { + pref = collectionPreferences.get(collection); + } + + if (pref > bestPref) { + bestPref = pref; + bestDuplicateIdx = i; + } + } + } + return bestDuplicateIdx; + } + + private List mergeKeywords(List allKeywords) { + Map, Pair, String>> keywordsMap = new HashMap, Pair, String>>(); + // type, lang, keywords, comment + + for (KeywordsList kwdList : allKeywords) { + Pair typeAndLang = new Pair(kwdList.getType(), + kwdList.getLanguage()); + Pair, String> keywordsAndComment; + String comment = kwdList.getComment(); + if (!keywordsMap.containsKey(typeAndLang)) { + keywordsAndComment = new Pair, String>( + new HashSet(), comment); + keywordsMap.put(typeAndLang, keywordsAndComment); + } else { + keywordsAndComment = keywordsMap.get(typeAndLang); + if (!comment.isEmpty()) { + String oldComment = keywordsAndComment.getY(); + if (oldComment.isEmpty()) { + keywordsAndComment.setY(comment); + } else { + keywordsAndComment.setY(oldComment + "\t" + comment); + } + } + } + keywordsAndComment.getX().addAll(kwdList.getKeywordsList()); + } + + List result = new ArrayList(); + for (Map.Entry, Pair, String>> entry : keywordsMap + .entrySet()) { + KeywordsList.Builder kwdlBuilder = KeywordsList.newBuilder(); + String type = entry.getKey().getX(); + String lang = entry.getKey().getY(); + Set keywords = entry.getValue().getX(); + String comment = entry.getValue().getY(); + + if (type != null && !type.isEmpty()) { + kwdlBuilder.setType(type); + } + if (lang != null && !lang.isEmpty()) { + kwdlBuilder.setLanguage(lang); + } + if (comment != null && !comment.isEmpty()) { + kwdlBuilder.setComment(comment); + } + kwdlBuilder.addAllKeywords(keywords); + + result.add(kwdlBuilder.build()); + } + return result; + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java new file mode 100644 index 00000000..dd4327de --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/DuplicatesMerger.java @@ -0,0 +1,13 @@ +package pl.edu.icm.coansys.document.deduplication.merge; + +import java.util.List; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; + +/** + * + * @author acz + */ +public interface DuplicatesMerger { + public void setup(String mergerConfiguration); + public DocumentWrapper merge(List duplicates); +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java new file mode 100644 index 00000000..2e01d455 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/merge/SimpleDuplicatesMerger.java @@ -0,0 +1,98 @@ +package pl.edu.icm.coansys.document.deduplication.merge; + +//import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.UUID; +import org.apache.commons.lang.StringUtils; +import pl.edu.icm.coansys.models.DocumentProtos.Author; +import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import pl.edu.icm.coansys.models.DocumentProtos.KeyValue; +//import pl.edu.icm.coansys.1output.merge.MergeConstants; + +/** + * Chooses first DocumentWrapper, updates keys in DocumentWrapper, DocumentMetadata and authors, gets extIds from all + * DocumentWrappers + * + * @author acz + */ +public class SimpleDuplicatesMerger implements DuplicatesMerger { + + @Override + public DocumentWrapper merge(List duplicates) { + + if (duplicates == null || duplicates.isEmpty()) { + throw new RuntimeException("Nothing to merge"); + } else if (duplicates.size() == 1) { + return duplicates.get(0); + } + + // Collect information for final result + List identifiers = new ArrayList(duplicates.size()); + Map> extIds = new HashMap>(); + SortedSet sortedCollections = new TreeSet(); + + for (DocumentWrapper dw : duplicates) { + DocumentMetadata dm = dw.getDocumentMetadata(); + identifiers.add(dw.getRowId()); + for (String collection : dm.getCollectionList()) { + sortedCollections.add(collection); + } + for (KeyValue id : dm.getExtIdList()) { + String idSource = id.getKey(); + String idValue = id.getValue(); + if (!extIds.containsKey(idSource)) { + extIds.put(idSource, new HashSet()); + } + extIds.get(idSource).add(idValue); + } + } + Collections.sort(identifiers); + String joinedIds = StringUtils.join(identifiers, "???");//MergeConstants.MERGED_ID_SEPARATOR); + String newIdentifier = UUID.nameUUIDFromBytes(joinedIds.getBytes()).toString(); + + // Create new DocumentWrapper.Builder + DocumentWrapper.Builder resultBuilder = DocumentWrapper.newBuilder(duplicates.get(0)); + + // Modify fields of DocumentWrapper.Builder + resultBuilder.setRowId(newIdentifier); + + DocumentMetadata.Builder documentMetadataBuilder = resultBuilder.getDocumentMetadataBuilder(); + + BasicMetadata.Builder basicMetadataBuilder = documentMetadataBuilder.getBasicMetadataBuilder(); + documentMetadataBuilder.setKey(newIdentifier); + documentMetadataBuilder.addAllCollection(sortedCollections); + + for (Author.Builder authorBuilder : basicMetadataBuilder.getAuthorBuilderList()) { + String positionSuffix = authorBuilder.getKey().replaceAll(".*(#c\\d+)", "$1"); + authorBuilder.setDocId(newIdentifier); + authorBuilder.setKey(newIdentifier + positionSuffix); + } + + documentMetadataBuilder.clearExtId(); + for (String eKey : extIds.keySet()) { + for (String eValue : extIds.get(eKey)) { + KeyValue.Builder ei = KeyValue.newBuilder(); + ei.setKey(eKey); + ei.setValue(eValue); + documentMetadataBuilder.addExtId(ei); + } + } + + // Build and return DocumentWrapper + return resultBuilder.build(); + } + + @Override + public void setup(String mergerConfiguration) { + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java new file mode 100644 index 00000000..5045a14c --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java @@ -0,0 +1,110 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.Arrays; +import static org.testng.Assert.*; +import org.testng.annotations.Test; + +/** + * + * @author Aleksander Nowinski + */ +public class CustomOddsCharsKeyGeneratorTest { + + public CustomOddsCharsKeyGeneratorTest() { + } + + public void setUp() { + } + + public void tearDown() { + } + + /** + * Test of generateKey method, of class CustomOddsCharsKeyGenerator. + */ + @Test + public void testGenerateKey() { + System.out.println("generateKey"); + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + + assertEquals(instance.getKeySize(), CustomOddsCharsKeyGenerator.DEFAULT_KEY_SIZE); + assertEquals(instance.generateKey(""), ""); + assertEquals(instance.generateKey("Short legged cat"), "soteg"); + assertEquals(instance.generateKey("a \t\t12 domino titles"), "1dmnt"); + instance.setKeySize(2); + assertEquals(instance.generateKey("The eleven elves"), "ee"); + assertEquals(instance.generateKey("404"), "44"); + instance.setKeySize(10); + assertEquals(instance.generateKey("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"), "bbbbbbbbbb"); + } + + @Test + public void testSetKeySizes() { + System.out.println("setKeySizes"); + + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + try { + instance.setKeySizes(null); + fail("Permitted null."); + } catch (IllegalArgumentException iae) { + //this is ok. + } + try { + instance.setKeySizes(new int[0]); + fail("Permitted empty array."); + } catch (IllegalArgumentException iae) { + //this is ok. + } + try { + instance.setKeySizes(new int[]{1, 2, 3, 3}); + fail("Permitted two equal params"); + } catch (IllegalArgumentException iae) { + //this is ok. + } + try { + instance.setKeySizes(new int[]{1, 2, 4, 3}); + fail("Permitted unsorted array"); + } catch (IllegalArgumentException iae) { + //this is ok. + } + int[] a = new int[] {1,3,6,7,8,9}; + instance.setKeySizes(a); + assertTrue(Arrays.equals(a, instance.getKeySizes())); + + } + + @Test + public void testGenerateKeys() { + System.out.println("generateKeys"); + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + instance.setKeySizes(new int[]{2, 3, 5}); + assertTrue(Arrays.equals(new String[]{"", "", ""}, instance.generateKeys(""))); + assertTrue(Arrays.equals(new String[]{"so", "sot", "soteg"}, instance.generateKeys("Short legged cat"))); + assertTrue(Arrays.equals(new String[]{"44", "44", "44"}, instance.generateKeys("404"))); + assertTrue(Arrays.equals(new String[]{"bb", "bbb", "bbbbb"}, instance.generateKeys("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"))); + } + + /** + * Test of cleanUpString method, of class CustomOddsCharsKeyGenerator. + */ + @Test + public void testCleanUpString() { + System.out.println("cleanUpString"); + CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); + assertEquals(instance.cleanUpString("test"), "test"); + assertEquals(instance.cleanUpString(".-- test --:+"), "test"); + assertEquals(instance.cleanUpString(".-- test -ab-:+"), "testab"); + assertEquals(instance.cleanUpString(".-- test 2-ab-:+"), "test2ab"); + assertEquals(instance.cleanUpString("\t\n test \t\ttest "), "testtest"); + assertEquals(instance.cleanUpString("test of cat"), "testcat"); + assertEquals(instance.cleanUpString("TeSt oF caT\t\t\n"), "testcat"); + assertEquals(instance.cleanUpString("Koń jak koń"), "konjakkon"); + assertEquals(instance.cleanUpString(" Litera β"), "literabeta"); + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml new file mode 100644 index 00000000..e1a28320 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml @@ -0,0 +1,35 @@ + + + deduplication-document + pl.edu.icm.coansys + 1.11-SNAPSHOT + + 4.0.0 + deduplication-document-spark-workflow + oozie + Deduplication - Document - SparkVersion - Workflow + + UTF-8 + + + + + pl.edu.icm.maven + oozie-maven-plugin + true + + + + + + pl.edu.icm.coansys + deduplication-document-impl + ${project.version} + + + pl.edu.icm.oozie + oozie-runner + test + + + diff --git a/deduplication-document-spark/pom.xml b/deduplication-document-spark/pom.xml new file mode 100644 index 00000000..b46619a7 --- /dev/null +++ b/deduplication-document-spark/pom.xml @@ -0,0 +1,20 @@ + + + 4.0.0 + + + pl.edu.icm.coansys + coansys + 1.11-SNAPSHOT + + + deduplication-document-spark + pom + Deduplication - Document SparkVersion + http://maven.apache.org + + + deduplication-document-spark-impl + deduplication-document-spark-workflow + + From c056a0b12aeddce50886a235df01ea57cfa4f8d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Tue, 4 Apr 2017 17:43:38 +0200 Subject: [PATCH 13/20] Work on the algorithm which splits large clusters among the cluster to obtain better scalability. --- .../deduplication/DeduplicateDocuments.scala | 328 +++++++++++++----- .../document/deduplication/TileTask.java | 128 +++++++ 2 files changed, 363 insertions(+), 93 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 72e33920..d339b6f1 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -22,123 +22,146 @@ import org.apache.spark.rdd.RDD import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer +import java.io.File +import java.io.PrintWriter +import pl.edu.icm.coansys.document.deduplication._ object DeduplicateDocuments { val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 5 - val maximumClusteringKeySize = 15 - val maximumClusterSize = 300 + val initialClusteringKeySize = 7 + val maximumClusteringKeySize = 18 + val maximumClusterSize = 600 //max author count - ignored // def calculateKey(doc: DocumentMetadata): String = { // new OddsCharsKeyGenerator().generateKey(doc) // } + def isValidDocument(doc: DocumentWrapper): Boolean = { + var res = false + if (doc.hasDocumentMetadata) { + val md = doc.getDocumentMetadata + if(md.hasBasicMetadata) { + val bmd = md.getBasicMetadata + if(bmd.getTitleCount()>0||bmd.getAuthorCount>0||bmd.hasDoi||bmd.hasJournal) { + res=true + } + } + } + res + } + def calculateKey(doc: DocumentMetadata, size: Int): String = { new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - def prepareClusters(inputDocs: RDD[DocumentWrapper]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Started preparation of clusters.") - var keySize = initialClusteringKeySize - - var approvedClusters: RDD[(String, Iterable[DocumentWrapper])] = null - var docs = inputDocs - var docCount = 0L - var iteration = 1 - - while ({ docCount = docs.count; docCount } > 0 && keySize < maximumClusteringKeySize) { - log.info("Starting iteration %d, keySize: %d, docs to count: %d".format(iteration, keySize, docCount)) - - var processedClusters = docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey() - var posClusters = processedClusters.filter(p => p._2.size <= maximumClusterSize) - if (approvedClusters == null) { - approvedClusters = posClusters - } else { - approvedClusters = approvedClusters.union(posClusters) - } - docs = processedClusters.filter(p => p._2.size > maximumClusterSize).flatMap(p => p._2) - keySize += 1 - iteration += 1 - } - log.info("Finished loop, keySize: %d, iterations done: %d, docs left: %d".format(keySize - 1, iteration - 1, docCount)) - if (docCount > 0) { - log.info("Adding leftovers (%d documents) with keySize %d".format(docCount, keySize)) - approvedClusters = approvedClusters.union(docs.map(doc => (calculateKey(doc.getDocumentMetadata, keySize), doc)).groupByKey()) + def calculateKeys(doc: DocumentMetadata): Array[String] = { + val keySizes = initialClusteringKeySize to maximumClusteringKeySize + val generator = new CustomOddsCharsKeyGenerator() + generator.setKeySizes(keySizes.toArray) + var res = generator.generateKeyList(doc) + //special case: if doc + if (res(0) == "") { + res = Array.fill[String](keySizes.length)(doc.getKey) } - approvedClusters + res } - /** - * @param args the command line arguments - */ - def main(args: Array[String]): Unit = { - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supported, exitting.") - return ; - } else { - println("Arguments:") - args.foreach(println); - } - println("Creating context...") - - val conf = new SparkConf() - .setAppName("Document deduplication") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") - - val sc = new SparkContext(conf) - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) - val outputDocuments = args(1) + def cleanKeyArray(in: Array[String]): Array[String] = { - // val debugDir = if (args.size > 2) Some(args(2)) else None + null + } - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - log.info("Loaded raw bytes.") - // rawbytes.count() - // log.info("Counted raw bytes.") + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Initializing cluster preparation (V2)") + val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) + + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + //build rdd (docId, (clusterId, clusterSize) ) + val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) + //build rdd - (docId, clusterId) + val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { + if (x._2 <= maximumClusterSize) { + if (y._2 <= maximumClusterSize) { + if (x._1.length <= y._1.length) { x } else { y } + } else { + x + } + } else { + if (y._2 <= maximumClusterSize) { + y + } else { + if (x._1.length > y._1.length) { x } else { y } + } + } + }).mapValues(_._1) - val wrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey + } - -// val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() - val initialGroups = prepareClusters(wrappers.map(_._2)) - log.info("After initial group preparation count.") - // val igs = initialGroups.count() - val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - // val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) - val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) - val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) - val timing = timedDeduplicated.map(x => x._2) + def prepareClusters(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Started preparation of clusters.") + var keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + var idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + var approvedClusters: RDD[(String, Iterable[String])] = null; + + //clusters to do are expected to have key as the first in key set. + var clustersToDo = idClusterKeys.map(p => (p._2.head, (p._1, p._2.drop(1)))) + do { + var currentKeySize = keySizes.head + keySizes = keySizes.drop(1) + log.info("Preparing clusters with key size: " + currentKeySize) + val newDivision = clustersToDo.groupByKey + log.info("Created new division, had " + clustersToDo.count + " document to group, built up " + newDivision.count + " clusters out of them.") + def clusterApproved(clusterId: String, cluster: Iterable[(String, Array[String])]): Boolean = { + (cluster.size <= maximumClusterSize) || (clusterId.length < currentKeySize) + } - log.info("After reducing clusters (comparison)") - val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) - log.info("Finished merge") - // val mergedSize = merged.count() + val goodClusters = if (keySizes.length > 0) + newDivision.filter(p => clusterApproved(p._1, p._2)) + else + newDivision - val single = initialGroups.filter(t => t._2.size == 1) - // log.info("Got initial group count=" + igs + "; singular=" + single.count + - // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) - // - //now merge the arrays: - val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) - if ("-" != outputDocuments) { - val bas = toWrite.mapValues(doc => doc.toByteArray()) - bas.saveAsSequenceFile(outputDocuments); - } - val tgrouped = timing.groupByKey; - val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect - println("================ Timing stats ======================") - stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) - println("================ end of timing stats ======================") + val goodClustersCleaned = goodClusters.mapValues(cl => cl.map(_._1)) - println("Exit") + if (approvedClusters == null) { + approvedClusters = goodClustersCleaned + } else { + approvedClusters = approvedClusters.union(goodClustersCleaned) + } + if (keySizes.length > 0) { + val unapprovedClusters = newDivision.filter(p => (!clusterApproved(p._1, p._2))) + clustersToDo = unapprovedClusters.flatMap(p => p._2.map(x => (x._2.head, (x._1, x._2.drop(1))))) + val sleft = clustersToDo.count + log.info("Finished cluster selection, good: " + goodClusters.count + "; too big: " + unapprovedClusters.count + " clusters.") + log.info("Finished iteration, " + sleft + " documents left.") + if (sleft == 0) { + log.info("Cluster preparation finished, all is OK.") + } + } else { + log.info("Finished last iteration.") + } + } while (keySizes.length > 0); + + log.info("Preparing docs again.") + //now join: + val doclClusterMap = approvedClusters.flatMap(v => v._2.map(x => (x, v._1))) + val totalDocs = doclClusterMap.count() + log.info("Finished flat map for join, docs: " + totalDocs) + val clusteringRes = doclClusterMap.join(inputDocs).map(x => x._2); + val clusteringResCount = clusteringRes.count + log.info("Finished join, got clustered docs: " + clusteringResCount) + + val res = clusteringRes.groupByKey() + val resSize = res.count + log.info("Finished cluster preparation, got clusters: " + resSize) + res } def buildDocumentsMerger(): DuplicatesMerger = { @@ -295,4 +318,123 @@ object DeduplicateDocuments { (rresult, (cluster.size, (rresult.size, end - start))) } + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + val enableClusterSummary = false; + val fixInvalidDocuments = true; + val removeDoubles=true; + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supported, exitting.") + return ; + } else { + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + val outputDocuments = args(1) + + // val debugDir = if (args.size > 2) Some(args(2)) else None + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + log.info("Loaded raw bytes.") + // rawbytes.count() + // log.info("Counted raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if(fixInvalidDocuments) { + val x = dirtyWrappers.filter(w=>isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info("Filtering invalid documents done, before filtering: "+preSize+" and after filtering "+afterSize+" documents left.") + x + } else { + dirtyWrappers + } + + val wrappers = if(removeDoubles) { + fixedWrappers.reduceByKey((x,y)=>y) + } else { + fixedWrappers + } + + + // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() + val initialGroups = prepareClustersV2(wrappers) + log.info("After initial group preparation count.") + + if (enableClusterSummary) { + //prepare cluster summary + + val largeClusters = initialGroups.filter(_._2.size > 1000) + + largeClusters.mapValues(_.size).take(250).foreach(println) + + val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect + + // val result = largeClusters.collect() + sampledResult.foreach(x => { + val docs = x._2._2 + val clusterId = x._1 + val fileBase = clusterId + "-%04d-".format(docs.size) + println(fileBase) + + println("Writing docs...") + var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") + docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) + pw.close + }) + return + } + + // val igs = initialGroups.count() + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + //val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) + val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) + val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) + val timing = timedDeduplicated.map(x => x._2) + + log.info("After reducing clusters (comparison)") + val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) + log.info("Finished merge") + // val mergedSize = merged.count() + + val single = initialGroups.filter(t => t._2.size == 1) + // log.info("Got initial group count=" + igs + "; singular=" + single.count + + // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) + // + //now merge the arrays: + val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) + if ("-" != outputDocuments) { + val bas = toWrite.mapValues(doc => doc.toByteArray()) + bas.saveAsSequenceFile(outputDocuments); + } else { + log.info("Simulating timing by counting.") + toWrite.count() + } + val tgrouped = timing.groupByKey; + val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect + println("================ Timing stats ======================") + stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) + println("================ end of timing stats ======================") + + println("Exit") + + } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java new file mode 100644 index 00000000..38887c09 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -0,0 +1,128 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.ListIterator; +import java.util.Set; +import java.util.function.BiPredicate; +import java.util.stream.Collectors; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import scala.Tuple2; + +/** + * + * @author Aleksander Nowinski + */ +public class TileTask { + String clusterId; + List rows; + List columns; + + public TileTask(String clusterId, List rows, List columns) { + this.clusterId = clusterId; + this.rows = rows; + this.columns = columns; + } + + + + public List> processPairs(BiPredicate x) { + List> res = new ArrayList<>(); + + + for (DocumentWrapper row : rows) { + for (DocumentWrapper column : columns) { + if(row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey())<0){ + if(x.test(row, column)) { + res.add(new Tuple2<>(row.getDocumentMetadata().getKey(), column.getDocumentMetadata().getKey())); + } + } + } + } + + return res; + } + + /** Prepare set of tile tasks representing task of comparing the cluster cartesian. It is + * natural to use this as a flatMap operator. + * + * @param clusterId id of the cluster added to the tasks + * @param docs list of documents to be cross-compared + * @param tileSize desired size of the single tile task + * @return list of tasks to be executed in parallel. + */ + public static List parallelize(String clusterId, Collection docs,int tileSize) { + List d = new ArrayList<>(docs); + Collections.sort(d, (d1, d2) -> d1.getDocumentMetadata().getKey().compareTo(d2.getDocumentMetadata().getKey())); + + int ntiles = docs.size()/tileSize+(docs.size()%tileSize==0?0:1); + List portions[] = new List[ntiles]; + for (int i = 0; i < d.size(); i++) { + int idx = i%ntiles; + if(portions[idx]==null) { + portions[idx]=new ArrayList(); + } + portions[idx].add(d.get(i)); + } + List res = new ArrayList<>(); + + + for (int i = 0; i < portions.length; i++) { + List rows = portions[i]; + for (int j = i; j < portions.length; j++) { + List columns = portions[j]; + if(rows!=null && columns!=null) { + res.add(new TileTask(clusterId, rows, columns)); + } + } + + } + return res; + } + + /** + * Method which merges clusters of the identifiers ensuring that afterwards + * all clusters where the same element appears are joined into one. It is + * intended to be used as a reduce operator. + * @param r1 first list of clusters + * @param r2 second list of clusters. + * @return list of clusters. + */ + public static List> coalesceResult(List> r1, List>r2) { + List> all = new ArrayList<>(); + all.addAll(remapToSets(r1)); + all.addAll(remapToSets(r2)); + List> res = new ArrayList<>(); + while(!all.isEmpty()) { + Set current = all.remove(0); + boolean anyChange = false; + do { + anyChange=false; + ListIterator> li = all.listIterator(); + while(li.hasNext()) { + Set next = li.next(); + if(next.stream().anyMatch(f->current.contains(f))) { + current.addAll(next); + li.remove(); + anyChange=true; + } + } + } while(anyChange); //necessary, as there may be chain of induced joins + res.add(new ArrayList(current)); + } + return res; + } + + protected static List> remapToSets(List> r2) { + return r2.stream().map(x->new HashSet(x)).collect(Collectors.toList()); + } + +} From e7ad7aa3b47e76db93acc5247f89281a88b92465 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 7 Apr 2017 15:54:11 +0200 Subject: [PATCH 14/20] Complete version with tiled comparison task. Added programatical logging into stdout, for easier log reading --- .../deduplication-document-spark-impl/pom.xml | 2 - .../deduplication/DeduplicateDocuments.scala | 749 +++++++++--------- .../document/deduplication/TileTask.java | 18 +- .../deduplication/TileTaskNGTest.java | 149 ++++ 4 files changed, 518 insertions(+), 400 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index 6461abb8..6d6263c9 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -21,7 +21,6 @@ src/main/scala src/test/scala - @@ -85,7 +84,6 @@ META-INF/*.SF META-INF/*.DSA META-INF/*.RSA - diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index d339b6f1..67294e0b 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -2,9 +2,12 @@ package pl.edu.icm.coansys.document.deduplication import scala.collection.JavaConversions._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import java.util.function.BiPredicate import org.apache.hadoop.io.BytesWritable +import org.apache.log4j.ConsoleAppender import org.apache.log4j.Level import org.apache.log4j.Logger +import org.apache.log4j.PatternLayout import org.apache.spark.SparkConf import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter @@ -19,422 +22,384 @@ import pl.edu.icm.coansys.document.deduplication.merge.DuplicatesMerger import pl.edu.icm.coansys.models.DocumentProtos import pl.edu.icm.coansys.models.DocumentProtos._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer import java.io.File import java.io.PrintWriter import pl.edu.icm.coansys.document.deduplication._ +import scala.collection.JavaConverters._ object DeduplicateDocuments { - val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 7 - val maximumClusteringKeySize = 18 - val maximumClusterSize = 600 - //max author count - ignored - - // def calculateKey(doc: DocumentMetadata): String = { - // new OddsCharsKeyGenerator().generateKey(doc) - // } - - def isValidDocument(doc: DocumentWrapper): Boolean = { - var res = false - if (doc.hasDocumentMetadata) { - val md = doc.getDocumentMetadata - if(md.hasBasicMetadata) { - val bmd = md.getBasicMetadata - if(bmd.getTitleCount()>0||bmd.getAuthorCount>0||bmd.hasDoi||bmd.hasJournal) { - res=true + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) + val initialClusteringKeySize = 7 + val maximumClusteringKeySize = 18 + val maximumClusterSize = 600 + val tileSize = 50 + + + def isValidDocument(doc: DocumentWrapper): Boolean = { + var res = false + if (doc.hasDocumentMetadata) { + val md = doc.getDocumentMetadata + if (md.hasBasicMetadata) { + val bmd = md.getBasicMetadata + if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { + res = true + } } } + res } - res - } - - def calculateKey(doc: DocumentMetadata, size: Int): String = { - new CustomOddsCharsKeyGenerator(size).generateKey(doc) - } - - def calculateKeys(doc: DocumentMetadata): Array[String] = { - val keySizes = initialClusteringKeySize to maximumClusteringKeySize - val generator = new CustomOddsCharsKeyGenerator() - generator.setKeySizes(keySizes.toArray) - var res = generator.generateKeyList(doc) - //special case: if doc - if (res(0) == "") { - res = Array.fill[String](keySizes.length)(doc.getKey) + + def calculateKey(doc: DocumentMetadata, size: Int): String = { + new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - res - } - - def cleanKeyArray(in: Array[String]): Array[String] = { - - null - } - - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Initializing cluster preparation (V2)") - val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray - log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved - val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) - - val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) - - //build rdd (docId, (clusterId, clusterSize) ) - val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) - //build rdd - (docId, clusterId) - val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { - if (x._2 <= maximumClusterSize) { - if (y._2 <= maximumClusterSize) { - if (x._1.length <= y._1.length) { x } else { y } - } else { - x - } - } else { - if (y._2 <= maximumClusterSize) { - y - } else { - if (x._1.length > y._1.length) { x } else { y } - } - } - }).mapValues(_._1) - - inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey - } - - def prepareClusters(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Started preparation of clusters.") - var keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray - log.info("Will use key sizes: " + keySizes.mkString(", ")) - var idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved - var approvedClusters: RDD[(String, Iterable[String])] = null; - - //clusters to do are expected to have key as the first in key set. - var clustersToDo = idClusterKeys.map(p => (p._2.head, (p._1, p._2.drop(1)))) - do { - var currentKeySize = keySizes.head - keySizes = keySizes.drop(1) - log.info("Preparing clusters with key size: " + currentKeySize) - val newDivision = clustersToDo.groupByKey - log.info("Created new division, had " + clustersToDo.count + " document to group, built up " + newDivision.count + " clusters out of them.") - def clusterApproved(clusterId: String, cluster: Iterable[(String, Array[String])]): Boolean = { - (cluster.size <= maximumClusterSize) || (clusterId.length < currentKeySize) - } - - val goodClusters = if (keySizes.length > 0) - newDivision.filter(p => clusterApproved(p._1, p._2)) - else - newDivision - - val goodClustersCleaned = goodClusters.mapValues(cl => cl.map(_._1)) - - if (approvedClusters == null) { - approvedClusters = goodClustersCleaned - } else { - approvedClusters = approvedClusters.union(goodClustersCleaned) - } - - if (keySizes.length > 0) { - val unapprovedClusters = newDivision.filter(p => (!clusterApproved(p._1, p._2))) - clustersToDo = unapprovedClusters.flatMap(p => p._2.map(x => (x._2.head, (x._1, x._2.drop(1))))) - val sleft = clustersToDo.count - log.info("Finished cluster selection, good: " + goodClusters.count + "; too big: " + unapprovedClusters.count + " clusters.") - log.info("Finished iteration, " + sleft + " documents left.") - if (sleft == 0) { - log.info("Cluster preparation finished, all is OK.") - } - } else { - log.info("Finished last iteration.") - } - } while (keySizes.length > 0); - - log.info("Preparing docs again.") - //now join: - val doclClusterMap = approvedClusters.flatMap(v => v._2.map(x => (x, v._1))) - val totalDocs = doclClusterMap.count() - log.info("Finished flat map for join, docs: " + totalDocs) - val clusteringRes = doclClusterMap.join(inputDocs).map(x => x._2); - val clusteringResCount = clusteringRes.count - log.info("Finished join, got clustered docs: " + clusteringResCount) - - val res = clusteringRes.groupByKey() - val resSize = res.count - log.info("Finished cluster preparation, got clusters: " + resSize) - res - } - - def buildDocumentsMerger(): DuplicatesMerger = { - val res = new AdvancedDuplicatesMerger - res.setup("") - res - } - - def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { - val merger = buildDocumentsMerger() - val merged = merger.merge(docs); - merged - } - - def buildWorkComparator(): WorkComparator = { - val result = new VotesProductComparator; - result.setMinVotersWeightRequired(1.5f) - result.setProbabilityTreshold(0.5f) - result.setTresholdIncreasingVotersRequired(0.7f) - - val voters = new ListBuffer[SimilarityVoter]() - val dv = new DoiVoter() - dv.setWeight(1.0f) - voters += dv - val jv = new JournalVoter() - jv.setWeight(0.3f) - jv.setDisapproveLevel(0.5f) - jv.setApproveLevel(0.05f) - voters += jv - - val wivv = new IssueVolumeVoter - wivv.setWeight(0.3f) - wivv.setAbstainIfAbsent(true) - wivv.setSubsetResult(0.8f) - wivv.setPartiallyMatchResult(0.52f) - voters += wivv - - val wpv = new PagesVoter - wpv.setWeight(.3f) - wpv.setAbstainIfAbsent(true) - wpv.setAbsentResult(0.6f) - wpv.setSubsetResult(0.75f) - wpv.setPartiallyMatchResult(0.64f) - wpv.setRemoveRepeated(true) - voters += wpv - - val wyv = new YearVoter - wyv.setWeight(.3f) - wyv.setAbstainIfAbsent(true) - wyv.setAbsentResult(.52f) - wyv.setSubsetResult(.9f) - wyv.setPartiallyMatchResult(.75f) - wyv.setRemoveRepeated(true) - voters += wyv - - val wtv = new TitleVoter() - wtv.setWeight(0.8f) - wtv.setDisapproveLevel(0.11f) - wtv.setApproveLevel(0.001f) - wtv.setMaxNormalizedTitleLength(90) - voters += wtv - - val wav = new AuthorsVoter - wav.setWeight(0.8f) - wav.setDisapproveLevel(0.2f) - wav.setApproveLevel(0.03f) - voters += wav - - result.setSimilarityVoters(voters) - result; - } - - //size, num, min time, max time, avg time, avg_group_count - def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { - val count = measurements.size - val times = measurements.map(_._2) - val minTime = times.min / 1000.0 - val maxTime = times.max / 1000.0 - val avgTime = times.sum / (1000.0 * count) - val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) - (size, count, minTime, maxTime, avgTime, avgGroupSize); - } - - def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { - log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) - - val res = Array.ofDim[Int](cluster.size, cluster.size) - val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) - val comparator = buildWorkComparator() - - val sind = sorted.toList.zipWithIndex - - val procStart = System.currentTimeMillis - sind.foreach(p1 => { - val i1 = p1._2 - val d1 = p1._1 - //make sure diagonal is zeroed - (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) - //simple iteration over table - sind.foreach(p2 => { - val i2 = p2._2 - val d2 = p2._1 - if (i1 < i2) { - val s = System.currentTimeMillis - comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) - val e = System.currentTimeMillis - res(i1)(i2) = (e - s).toInt + + def calculateKeys(doc: DocumentMetadata): Array[String] = { + val keySizes = initialClusteringKeySize to maximumClusteringKeySize + val generator = new CustomOddsCharsKeyGenerator() + generator.setKeySizes(keySizes.toArray) + var res = generator.generateKeyList(doc) + //special case: if doc + if (res(0) == "") { + res = Array.fill[String](keySizes.length)(doc.getKey) } - }) - }) - val procEnd = System.currentTimeMillis - val elapsedSteps = res.map(_.sum).sum - log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) - - (sorted, res) - } - - //todo: timing depending on size - //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which - //are assumed to be duplicates. - def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { - log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) - // - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - if (cluster.size > 30) { - - log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) + res } - val startTime = System.currentTimeMillis(); - val classes = ListBuffer[ListBuffer[DocumentWrapper]]() - val comparator = buildWorkComparator() - cluster.foreach(x => { - val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) - if (arr.isEmpty) { - val nclass = ListBuffer(x) - classes += nclass - } else { - arr.get += x - } - }) - val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 - val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) - log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") - - res - - } - - def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { - val start = System.currentTimeMillis; - val rresult = deduplicateCluster(cluster, clusterId); - val end = System.currentTimeMillis - - (rresult, (cluster.size, (rresult.size, end - start))) - } - - /** - * @param args the command line arguments - */ - def main(args: Array[String]): Unit = { - val enableClusterSummary = false; - val fixInvalidDocuments = true; - val removeDoubles=true; - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supported, exitting.") - return ; - } else { - println("Arguments:") - args.foreach(println); + + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Initializing cluster preparation (V2)") + val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) + + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + //build rdd (docId, (clusterId, clusterSize) ) + val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) + //build rdd - (docId, clusterId) + val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { + if (x._2 <= maximumClusterSize) { + if (y._2 <= maximumClusterSize) { + if (x._1.length <= y._1.length) { x } else { y } + } else { + x + } + } else { + if (y._2 <= maximumClusterSize) { + y + } else { + if (x._1.length > y._1.length) { x } else { y } + } + } + }).mapValues(_._1) + inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey } - println("Creating context...") - val conf = new SparkConf() - .setAppName("Document deduplication") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + def buildDocumentsMerger(): DuplicatesMerger = { + val res = new AdvancedDuplicatesMerger + res.setup("") + res + } - val sc = new SparkContext(conf) - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" - val outputDocuments = args(1) + def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { + val merger = buildDocumentsMerger() + val merged = merger.merge(docs); + merged + } - // val debugDir = if (args.size > 2) Some(args(2)) else None + def buildWorkComparator(): WorkComparator = { + val result = new VotesProductComparator; + result.setMinVotersWeightRequired(1.5f) + result.setProbabilityTreshold(0.5f) + result.setTresholdIncreasingVotersRequired(0.7f) + + val voters = new ListBuffer[SimilarityVoter]() + val dv = new DoiVoter() + dv.setWeight(1.0f) + voters += dv + val jv = new JournalVoter() + jv.setWeight(0.3f) + jv.setDisapproveLevel(0.5f) + jv.setApproveLevel(0.05f) + voters += jv + + val wivv = new IssueVolumeVoter + wivv.setWeight(0.3f) + wivv.setAbstainIfAbsent(true) + wivv.setSubsetResult(0.8f) + wivv.setPartiallyMatchResult(0.52f) + voters += wivv + + val wpv = new PagesVoter + wpv.setWeight(.3f) + wpv.setAbstainIfAbsent(true) + wpv.setAbsentResult(0.6f) + wpv.setSubsetResult(0.75f) + wpv.setPartiallyMatchResult(0.64f) + wpv.setRemoveRepeated(true) + voters += wpv + + val wyv = new YearVoter + wyv.setWeight(.3f) + wyv.setAbstainIfAbsent(true) + wyv.setAbsentResult(.52f) + wyv.setSubsetResult(.9f) + wyv.setPartiallyMatchResult(.75f) + wyv.setRemoveRepeated(true) + voters += wyv + + val wtv = new TitleVoter() + wtv.setWeight(0.8f) + wtv.setDisapproveLevel(0.11f) + wtv.setApproveLevel(0.001f) + wtv.setMaxNormalizedTitleLength(90) + voters += wtv + + val wav = new AuthorsVoter + wav.setWeight(0.8f) + wav.setDisapproveLevel(0.2f) + wav.setApproveLevel(0.03f) + voters += wav + + result.setSimilarityVoters(voters) + result; + } - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - log.info("Loaded raw bytes.") - // rawbytes.count() - // log.info("Counted raw bytes.") + //size, num, min time, max time, avg time, avg_group_count + def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { + val count = measurements.size + val times = measurements.map(_._2) + val minTime = times.min / 1000.0 + val maxTime = times.max / 1000.0 + val avgTime = times.sum / (1000.0 * count) + val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) + (size, count, minTime, maxTime, avgTime, avgGroupSize); + } - val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) - - //fix invalid documents: - val fixedWrappers = if(fixInvalidDocuments) { - val x = dirtyWrappers.filter(w=>isValidDocument(w._2)) - val afterSize = x.count; - val preSize = dirtyWrappers.count - log.info("Filtering invalid documents done, before filtering: "+preSize+" and after filtering "+afterSize+" documents left.") - x - } else { - dirtyWrappers - } - - val wrappers = if(removeDoubles) { - fixedWrappers.reduceByKey((x,y)=>y) - } else { - fixedWrappers + def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { + log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) + + val res = Array.ofDim[Int](cluster.size, cluster.size) + val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) + val comparator = buildWorkComparator() + + val sind = sorted.toList.zipWithIndex + + val procStart = System.currentTimeMillis + sind.foreach(p1 => { + val i1 = p1._2 + val d1 = p1._1 + //make sure diagonal is zeroed + (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) + //simple iteration over table + sind.foreach(p2 => { + val i2 = p2._2 + val d2 = p2._1 + if (i1 < i2) { + val s = System.currentTimeMillis + comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) + val e = System.currentTimeMillis + res(i1)(i2) = (e - s).toInt + } + }) + }) + val procEnd = System.currentTimeMillis + val elapsedSteps = res.map(_.sum).sum + log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) + + (sorted, res) } - - - // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() - val initialGroups = prepareClustersV2(wrappers) - log.info("After initial group preparation count.") - - if (enableClusterSummary) { - //prepare cluster summary - - val largeClusters = initialGroups.filter(_._2.size > 1000) - - largeClusters.mapValues(_.size).take(250).foreach(println) - - val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect - - // val result = largeClusters.collect() - sampledResult.foreach(x => { - val docs = x._2._2 - val clusterId = x._1 - val fileBase = clusterId + "-%04d-".format(docs.size) - println(fileBase) - - println("Writing docs...") - var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") - docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) - pw.close - }) - return + + //todo: timing depending on size + //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which + //are assumed to be duplicates. + def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { + log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) + // + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + if (cluster.size > 30) { + + log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) + } + val startTime = System.currentTimeMillis(); + val classes = ListBuffer[ListBuffer[DocumentWrapper]]() + val comparator = buildWorkComparator() + cluster.foreach(x => { + val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) + if (arr.isEmpty) { + val nclass = ListBuffer(x) + classes += nclass + } else { + arr.get += x + } + }) + val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 + val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) + log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") + + res + } - // val igs = initialGroups.count() - val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - //val deduplicatedClusters = clustersToDeduplicate.flatMap(x => deduplicateCluster(x._2, x._1)) - val timedDeduplicated = clustersToDeduplicate.map(x => timedDeduplicateCluster(x._2, x._1)) - val deduplicatedClusters = timedDeduplicated.flatMap(x => x._1) - val timing = timedDeduplicated.map(x => x._2) - - log.info("After reducing clusters (comparison)") - val merged = deduplicatedClusters.map(x => (mergeDocuments(x._2))) - log.info("Finished merge") - // val mergedSize = merged.count() - - val single = initialGroups.filter(t => t._2.size == 1) - // log.info("Got initial group count=" + igs + "; singular=" + single.count + - // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) - // - //now merge the arrays: - val toWrite = merged.union(single.map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) - if ("-" != outputDocuments) { - val bas = toWrite.mapValues(doc => doc.toByteArray()) - bas.saveAsSequenceFile(outputDocuments); - } else { - log.info("Simulating timing by counting.") - toWrite.count() + def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { + val start = System.currentTimeMillis; + val rresult = deduplicateCluster(cluster, clusterId); + val end = System.currentTimeMillis + + (rresult, (cluster.size, (rresult.size, end - start))) } - val tgrouped = timing.groupByKey; - val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect - println("================ Timing stats ======================") - stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) - println("================ end of timing stats ======================") + implicit def toJavaBiPredicate[A, B](predicate: (A, B) ⇒ Boolean) = + new BiPredicate[A, B] { + def test(a: A, b: B) = predicate(a, b) + } + - println("Exit") + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + val enableClusterSummary = false; + val fixInvalidDocuments = true; + val removeDoubles = true; - } + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supported, exitting.") + return ; + } else { + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender + console.activateOptions(); + Logger.getLogger("pl.edu.icm").addAppender(console) + log.info("Reconfigured logger...") + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + val outputDocuments = args(1) + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + log.info("Loaded raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if (fixInvalidDocuments) { + val x = dirtyWrappers.filter(w => isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info("Filtering invalid documents done, before filtering: " + preSize + " and after filtering " + afterSize + " documents left.") + x + } else { + dirtyWrappers + } + + val wrappers = if (removeDoubles) { + fixedWrappers.reduceByKey((x, y) => y) + } else { + fixedWrappers + } + wrappers.persist(StorageLevel.MEMORY_AND_DISK) + + + // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() + val initialGroups = prepareClustersV2(wrappers) + initialGroups.persist(StorageLevel.MEMORY_AND_DISK) + + if (enableClusterSummary) { + //prepare cluster summary + val largeClusters = initialGroups.filter(_._2.size > 1000) + largeClusters.mapValues(_.size).take(250).foreach(println) + val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect + // val result = largeClusters.collect() + sampledResult.foreach(x => { + val docs = x._2._2 + val clusterId = x._1 + val fileBase = clusterId + "-%04d-".format(docs.size) + println(fileBase) + + println("Writing docs...") + var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") + docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) + pw.close + }) + return + } + + // val igs = initialGroups.count() + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + + val tiledTasks = clustersToDeduplicate.flatMap( + p => TileTask.parallelize(p._1, p._2, tileSize).map(v => (p._1, v)) + ) + + + + val equalityClusters = tiledTasks.flatMapValues( + v => { + val comparator = buildWorkComparator + v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => + comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) + } + ).mapValues(v => List(v._1, v._2)) + val allItemsTraces = initialGroups.flatMapValues(l => l.map(v => List(v.getDocumentMetadata.getKey))) + val finalClusters = equalityClusters.union(allItemsTraces).mapValues(List(_)). + reduceByKey( + (a, b) => TileTask.coalesceResult(a.map(x => x.asJava).asJava, b.map(x => x.asJava).asJava). + asScala.toList.map(x => x.asScala.toList) + ).flatMap( + p => { + val cid = p._1 + val cl = p._2 + cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) + }) + val finalDocClusters = finalClusters.flatMapValues(x => x). + map(v => (v._2, v._1)).join(wrappers).map(v => (v._2._1, v._2._2)). + groupByKey() + + val singularClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size==1) + val pluralClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size>1) + + log.info("After reducing clusters (comparison)") + val merged = pluralClustersAfterDeduplication.map(x => (mergeDocuments(x._2.toList))) + log.info("Finished merge") + // val mergedSize = merged.count() + + val single = initialGroups.filter(t => t._2.size == 1) + // log.info("Got initial group count=" + igs + "; singular=" + single.count + + // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) + // + //now merge the arrays: + val toWrite = merged.union(single.union(singularClustersAfterDeduplication).map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) + if ("-" != outputDocuments) { + val bas = toWrite.mapValues(doc => doc.toByteArray()) + bas.saveAsSequenceFile(outputDocuments); + } else { + log.info("Simulating timing by counting.") + toWrite.count() + } +// val tgrouped = timing.groupByKey; +// val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect +// println("================ Timing stats ======================") +// stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) +// println("================ end of timing stats ======================") + + println("Exit") + + } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java index 38887c09..61caf991 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -5,6 +5,7 @@ */ package pl.edu.icm.coansys.document.deduplication; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -21,7 +22,7 @@ * * @author Aleksander Nowinski */ -public class TileTask { +public class TileTask implements Serializable{ String clusterId; List rows; List columns; @@ -33,15 +34,20 @@ public TileTask(String clusterId, List rows, List> processPairs(BiPredicate x) { + /** + * Generate list of pairs of the documents, where predicate is conformed, ie function + * passed returned true. The predicate is assumed to be symmetrical, so it is executed + * only once on each pair. + * + * @param equalityTest predicate which defines whether or no two elements are considered matching (typically equal) + * @return list of pairs of keys of equal documents (documents where equalityTest returned true) + */ + public List> processPairs(BiPredicate equalityTest) { List> res = new ArrayList<>(); - - for (DocumentWrapper row : rows) { for (DocumentWrapper column : columns) { if(row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey())<0){ - if(x.test(row, column)) { + if(equalityTest.test(row, column)) { res.add(new Tuple2<>(row.getDocumentMetadata().getKey(), column.getDocumentMetadata().getKey())); } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java new file mode 100644 index 00000000..5f67425d --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java @@ -0,0 +1,149 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; +import com.google.common.collect.Range; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.function.BiPredicate; +import java.util.stream.Collectors; +import static org.testng.Assert.*; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import pl.edu.icm.coansys.models.DocumentProtos; +import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; +import scala.Tuple2; + +/** + * + * @author Aleksander Nowinski + */ +public class TileTaskNGTest { + + public TileTaskNGTest() { + } + + @BeforeMethod + public void setUpMethod() throws Exception { + } + + @AfterMethod + public void tearDownMethod() throws Exception { + } + + public DocumentWrapper createDocument(String key, String title) { + return DocumentWrapper.newBuilder().setDocumentMetadata( + DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( + BasicMetadata.newBuilder().addTitle( + DocumentProtos.TextWithLanguage.newBuilder().setText(title))) + ).setRowId(key).build(); + } + + protected List createDocumentList(int size) { + return ContiguousSet.create(Range.closed(1, size), DiscreteDomain.integers()).stream() + .map(i -> createDocument(String.format("key_%03d", i), String.format("title_%03d", i))) + .collect(Collectors.toList()); + } + + /** + * Test of processPairs method, of class TileTask. + */ + @Test + public void testProcessPairs() { + System.out.println("processPairs"); + int size = 500; + List documents = createDocumentList(size); + System.out.println("Generated documents:"); + System.out.println(documents.stream().limit(10).map(v -> v.toString()).reduce(((s1, s2) -> s1 + "\n" + s2))); + + TileTask instance = new TileTask("test", documents, documents); + List> res = instance.processPairs((a, b) -> false); + assertEquals(res.size(), 0); + + res = instance.processPairs((a, b) -> true); + assertEquals(res.size(), size * (size - 1) / 2); + + res = instance.processPairs( + (a, b) -> { + String k1 = a.getDocumentMetadata().getKey(); + String k2 = b.getDocumentMetadata().getKey(); + String k1s = k1.substring(5); + String k2s = k2.substring(5); + return k1s.equals(k2s); + }); +// System.out.println("Generated pairs:"); +// System.out.println(res.stream().map(v -> v.toString()).reduce(((s1, s2) -> s1 + ", " + s2))); + assertEquals(res.size(), 1000); + + } + + + private Set toKeySet(List wrappers) { + return wrappers.stream().map(x->x.getDocumentMetadata().getKey()).collect(Collectors.toSet()); + } + + /** + * Test of parallelize method, of class TileTask. + */ + @Test + public void testParallelize() { + System.out.println("parallelize"); + List docs = createDocumentList(9); + String clusterId = "cluster"; + List res = TileTask.parallelize(clusterId, docs, 200); + assertEquals(res.size(), 1, "Created too many tasks."); + + res = TileTask.parallelize(clusterId, docs, 5); + assertEquals(res.size(), 3, "Created invalid number of tasks."); + Set tile0r = toKeySet(res.get(0).rows); + Set tile0c = toKeySet(res.get(0).columns); + assertEquals(tile0r, tile0c); + + Set tile1r = toKeySet(res.get(1).rows); + Set tile1c = toKeySet(res.get(1).columns); + tile1r.stream().forEach( + key->assertFalse(tile1c.contains(key),"In tile 1 key "+key+" from row appears in columns.") + ); + + res = TileTask.parallelize(clusterId, docs, 2); + assertEquals(res.size(), 15, "Created invalid number tasks."); + res = TileTask.parallelize(clusterId, docs, 1); + assertEquals(res.size(), 45, "Created too many tasks."); + } + + /** + * Test of coalesceResult method, of class TileTask. + */ + @Test + public void testCoalesceResult() { + System.out.println("coalesceResult"); + List> r1 = Arrays.asList(new List[] { + Arrays.asList(new String[]{"a", "b"}), + Arrays.asList(new String[]{"c", "d"}), + Arrays.asList(new String[]{"e", "f"}), + }); + List> r2 = Arrays.asList(new List[] { + Arrays.asList(new String[]{"a", "c"}), + }); + + List> result = TileTask.coalesceResult(r1, r2); + assertEquals(2, result.size()); + List l1 = result.get(0); + Collections.sort(l1); + assertEquals(l1, + Arrays.asList("a", "b", "c", "d")); + } + + +} From 0cf6672e5839b01a59de478032f10d873b2f2dd5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 14 Apr 2017 12:10:41 +0200 Subject: [PATCH 15/20] Work on tiled optimalization. Added reshuffle for better work balance. --- .../deduplication/DeduplicateDocuments.scala | 104 ++++++++--- .../ExecuteOnceOnExecutorsHelpers.java | 40 ++++ .../document/deduplication/TileTask.java | 173 ++++++++++++------ .../deduplication/TileTaskNGTest.java | 27 ++- 4 files changed, 254 insertions(+), 90 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 67294e0b..cb696062 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -8,7 +8,9 @@ import org.apache.log4j.ConsoleAppender import org.apache.log4j.Level import org.apache.log4j.Logger import org.apache.log4j.PatternLayout +import org.apache.spark.HashPartitioner import org.apache.spark.SparkConf +import org.apache.spark.api.java.function.VoidFunction import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter import pl.edu.icm.coansys.deduplication.document.voter.IssueVolumeVoter @@ -36,7 +38,7 @@ object DeduplicateDocuments { val initialClusteringKeySize = 7 val maximumClusteringKeySize = 18 val maximumClusterSize = 600 - val tileSize = 50 + val tileSize = 30 def isValidDocument(doc: DocumentWrapper): Boolean = { @@ -252,12 +254,33 @@ object DeduplicateDocuments { (rresult, (cluster.size, (rresult.size, end - start))) } - implicit def toJavaBiPredicate[A, B](predicate: (A, B) ⇒ Boolean) = + + implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = new BiPredicate[A, B] { def test(a: A, b: B) = predicate(a, b) } - - + +// +// implicit def toVoidFunction[T](runMethod: (T) => Unit) = +// new VoidFunction[T]() { +// def call(t:T) = runMethod(t) +// } + + + def reconfigureLogging(sc: SparkContext) = { + ExecuteOnceOnExecutorsHelpers.executeEverywhere(sc, "RECONFIGURE_LOGGING", true, + new VoidFunction[java.util.Iterator[_]]() { + def call(t :java.util.Iterator[_]) :Unit={ + Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) + val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender + console.activateOptions(); + Logger.getLogger("pl.edu.icm").addAppender(console) + log.info("Reconfigured logger...") + } + } + ) + } + /** * @param args the command line arguments */ @@ -283,11 +306,8 @@ object DeduplicateDocuments { .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") val sc = new SparkContext(conf) - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender - console.activateOptions(); - Logger.getLogger("pl.edu.icm").addAppender(console) - log.info("Reconfigured logger...") + reconfigureLogging(sc) + println("Created context...") sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" @@ -303,7 +323,7 @@ object DeduplicateDocuments { val x = dirtyWrappers.filter(w => isValidDocument(w._2)) val afterSize = x.count; val preSize = dirtyWrappers.count - log.info("Filtering invalid documents done, before filtering: " + preSize + " and after filtering " + afterSize + " documents left.") + log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") x } else { dirtyWrappers @@ -315,11 +335,12 @@ object DeduplicateDocuments { fixedWrappers } wrappers.persist(StorageLevel.MEMORY_AND_DISK) + val initialSize = wrappers.count + log.info(f"Starting processing with $initialSize documents.") - - // val initialGroups = wrappers.map(t => (calculateKey(t._2.getDocumentMetadata, initialClusteringKeySize), t._2)).groupByKey() val initialGroups = prepareClustersV2(wrappers) initialGroups.persist(StorageLevel.MEMORY_AND_DISK) + if (enableClusterSummary) { //prepare cluster summary @@ -343,31 +364,54 @@ object DeduplicateDocuments { // val igs = initialGroups.count() val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - - val tiledTasks = clustersToDeduplicate.flatMap( - p => TileTask.parallelize(p._1, p._2, tileSize).map(v => (p._1, v)) - ) - - - - val equalityClusters = tiledTasks.flatMapValues( - v => { - val comparator = buildWorkComparator - v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => - comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) - } - ).mapValues(v => List(v._1, v._2)) - val allItemsTraces = initialGroups.flatMapValues(l => l.map(v => List(v.getDocumentMetadata.getKey))) - val finalClusters = equalityClusters.union(allItemsTraces).mapValues(List(_)). + val initialClusterCount = clustersToDeduplicate.count + + val tiledTasks = clustersToDeduplicate.flatMap( p => TileTask.parallelize(p._1, p._2, tileSize) ) + val shuffledTiledTasks = tiledTasks.repartition(5000); + shuffledTiledTasks.persist(StorageLevel.MEMORY_AND_DISK) + val tileCount = shuffledTiledTasks.count; + + log.info(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") +// val comparator = sc.broadcast(buildWorkComparator) + reconfigureLogging(sc) + val partialEqualityClusters = shuffledTiledTasks.flatMap( + v => { + val t0=java.lang.System.currentTimeMillis() + log.info("Starting tile task %s".format(v.getTaskId)) + val comparator = buildWorkComparator + val res = v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => + comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) + val time = (java.lang.System.currentTimeMillis()-t0)/1000. + log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") + res + } + ) +// equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) +// log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); +// val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) +// log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); +// +// val eqclsizes= filteredSizes.collect.sortBy (_._2) +// println("Equality cluster sizes:") +// eqclsizes.foreach(println(_)) +// println("Done.\n\n") + + val finalClusters = partialEqualityClusters.mapValues(List(_)). reduceByKey( - (a, b) => TileTask.coalesceResult(a.map(x => x.asJava).asJava, b.map(x => x.asJava).asJava). - asScala.toList.map(x => x.asScala.toList) + (a, b) => TileTask.coalesceResult(a, b).toList ).flatMap( p => { val cid = p._1 val cl = p._2 cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) }) + + finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); + val finclSizes=finalClusters.mapValues(_.size).filter(_._2>500).collect.sortBy(_._1) + println("\n\nFinal cluster sizes:") + finclSizes.foreach(println(_)) + println("Done.") + val finalDocClusters = finalClusters.flatMapValues(x => x). map(v => (v._2, v._1)).join(wrappers).map(v => (v._2._1, v._2._2)). groupByKey() diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java new file mode 100644 index 00000000..a413ad20 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java @@ -0,0 +1,40 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ +package pl.edu.icm.coansys.document.deduplication; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.VoidFunction; + +/** + * + * @author Aleksander Nowinski + */ +public class ExecuteOnceOnExecutorsHelpers { + + private static Set executedCode = new HashSet<>(); + + public static void executeEverywhere(SparkContext sc, String codeId, boolean runOnDriver, VoidFunction function) throws Exception{ + JavaSparkContext.fromSparkContext(sc).parallelize(Arrays.asList(new Integer[1000])).foreachPartition(new VoidFunction>() { + @Override + public void call(Iterator it) throws Exception { + if (!executedCode.contains(codeId)) { + ExecuteOnceOnExecutorsHelpers.executedCode.add(codeId); + function.call(it); + } + } + }); + if (runOnDriver) { + function.call(null); + } + } + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java index 61caf991..4af0524e 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -12,6 +12,7 @@ import java.util.HashSet; import java.util.List; import java.util.ListIterator; +import java.util.Objects; import java.util.Set; import java.util.function.BiPredicate; import java.util.stream.Collectors; @@ -22,8 +23,10 @@ * * @author Aleksander Nowinski */ -public class TileTask implements Serializable{ +public class TileTask implements Serializable { + String clusterId; + String taskId; List rows; List columns; @@ -32,103 +35,165 @@ public TileTask(String clusterId, List rows, List> processPairs(BiPredicate equalityTest) { - List> res = new ArrayList<>(); + * Generate list of pairs of the documents, where predicate is conformed, ie + * function passed returned true. The predicate is assumed to be + * symmetrical, so it is executed only once on each pair. + * + * @param equalityTest predicate which defines whether or no two elements + * are considered matching (typically equal) + * @return list of pairs of keys of equal documents (documents where + * equalityTest returned true), wrappet into a tuple with cluster id. + */ + public List>> processPairs(BiPredicate equalityTest) { + List> raw = new ArrayList<>(); for (DocumentWrapper row : rows) { - for (DocumentWrapper column : columns) { - if(row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey())<0){ - if(equalityTest.test(row, column)) { - res.add(new Tuple2<>(row.getDocumentMetadata().getKey(), column.getDocumentMetadata().getKey())); - } - } - } + final List rlist = new ArrayList<>(); + rlist.add(row.getDocumentMetadata().getKey()); + List equalColums = columns.stream() + .filter( + column -> (row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey()) < 0) + && (equalityTest.test(row, column)) + ).map(column -> column.getDocumentMetadata().getKey()) + .collect(Collectors.toList()); + rlist.addAll(equalColums); + raw.add(rlist); } - - return res; + raw = coalesceResult(raw); + return raw.stream().map(list -> new Tuple2<>(clusterId, list)).collect(Collectors.toList()); } - - /** Prepare set of tile tasks representing task of comparing the cluster cartesian. It is - * natural to use this as a flatMap operator. - * + + /** + * Prepare set of tile tasks representing task of comparing the cluster + * cartesian. It is natural to use this as a flatMap operator. + * * @param clusterId id of the cluster added to the tasks * @param docs list of documents to be cross-compared * @param tileSize desired size of the single tile task * @return list of tasks to be executed in parallel. */ - public static List parallelize(String clusterId, Collection docs,int tileSize) { + public static List parallelize(String clusterId, Collection docs, int tileSize) { List d = new ArrayList<>(docs); Collections.sort(d, (d1, d2) -> d1.getDocumentMetadata().getKey().compareTo(d2.getDocumentMetadata().getKey())); - - int ntiles = docs.size()/tileSize+(docs.size()%tileSize==0?0:1); + + int ntiles = docs.size() / tileSize + (docs.size() % tileSize == 0 ? 0 : 1); List portions[] = new List[ntiles]; for (int i = 0; i < d.size(); i++) { - int idx = i%ntiles; - if(portions[idx]==null) { - portions[idx]=new ArrayList(); + int idx = i % ntiles; + if (portions[idx] == null) { + portions[idx] = new ArrayList(); } portions[idx].add(d.get(i)); } List res = new ArrayList<>(); - - + for (int i = 0; i < portions.length; i++) { List rows = portions[i]; for (int j = i; j < portions.length; j++) { List columns = portions[j]; - if(rows!=null && columns!=null) { - res.add(new TileTask(clusterId, rows, columns)); + if (rows != null && columns != null) { + final TileTask ntask = new TileTask(clusterId, rows, columns); + ntask.taskId = String.format("%s_%04d:%04d", clusterId, i, j); + res.add(ntask); } } - + } return res; } - + /** - * Method which merges clusters of the identifiers ensuring that afterwards - * all clusters where the same element appears are joined into one. It is - * intended to be used as a reduce operator. - * @param r1 first list of clusters - * @param r2 second list of clusters. - * @return list of clusters. + * Combine clusters which have non-empty intersection, so result will be + * only separate lists. + * + * @param clusters lists to combine + * @return list of the separate clusters, as */ - public static List> coalesceResult(List> r1, List>r2) { + public static List> coalesceResult(List> clusters) { List> all = new ArrayList<>(); - all.addAll(remapToSets(r1)); - all.addAll(remapToSets(r2)); + all.addAll(remapToSets(clusters)); List> res = new ArrayList<>(); - while(!all.isEmpty()) { + while (!all.isEmpty()) { Set current = all.remove(0); boolean anyChange = false; do { - anyChange=false; + anyChange = false; ListIterator> li = all.listIterator(); - while(li.hasNext()) { + while (li.hasNext()) { Set next = li.next(); - if(next.stream().anyMatch(f->current.contains(f))) { + if (next.stream().anyMatch(f -> current.contains(f))) { current.addAll(next); li.remove(); - anyChange=true; + anyChange = true; } } - } while(anyChange); //necessary, as there may be chain of induced joins + } while (anyChange); //necessary, as there may be chain of induced joins res.add(new ArrayList(current)); } return res; } + /** + * Method which merges clusters of the identifiers ensuring that afterwards + * all clusters where the same element appears are joined into one. It is + * intended to be used as a reduce operator. + * + * @param r1 first list of clusters + * @param r2 second list of clusters. + * @return list of clusters. + */ + public static List> coalesceResult(List> r1, List> r2) { + List> all = new ArrayList<>(); + all.addAll(r1); + all.addAll(r2); + return coalesceResult(all); + } + protected static List> remapToSets(List> r2) { - return r2.stream().map(x->new HashSet(x)).collect(Collectors.toList()); + return r2.stream().map(x -> new HashSet<>(x)).collect(Collectors.toList()); } - + } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java index 5f67425d..9b3ebd49 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java @@ -9,11 +9,9 @@ import com.google.common.collect.DiscreteDomain; import com.google.common.collect.Range; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Set; -import java.util.function.BiPredicate; import java.util.stream.Collectors; import static org.testng.Assert.*; import org.testng.annotations.AfterMethod; @@ -68,11 +66,20 @@ public void testProcessPairs() { System.out.println(documents.stream().limit(10).map(v -> v.toString()).reduce(((s1, s2) -> s1 + "\n" + s2))); TileTask instance = new TileTask("test", documents, documents); - List> res = instance.processPairs((a, b) -> false); - assertEquals(res.size(), 0); + List>> res = instance.processPairs((a, b) -> false); + + assertEquals(res.size(), size); + assertTrue(res.stream().anyMatch(p->p._2.size()==1)); + assertEquals(res.stream().map(p->p._2.get(0)).distinct().count(), size); + res = instance.processPairs((a, b) -> true); - assertEquals(res.size(), size * (size - 1) / 2); + assertEquals(res.size(), 1); + assertEquals(res.get(0)._2.size(), size); + + + + assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); res = instance.processPairs( (a, b) -> { @@ -84,7 +91,15 @@ public void testProcessPairs() { }); // System.out.println("Generated pairs:"); // System.out.println(res.stream().map(v -> v.toString()).reduce(((s1, s2) -> s1 + ", " + s2))); - assertEquals(res.size(), 1000); + assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); + assertEquals(res.size(), 100); + + res.stream().map(x->x._2).forEach( + x->{ + String ke = x.get(0).substring(5); + assertTrue(x.stream().allMatch(key->key.endsWith(ke))); + } + ); } From 013b53c5c7276ca8f8f8292d66fc7f93cbb72c63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Fri, 23 Jun 2017 10:34:16 +0200 Subject: [PATCH 16/20] Stable version, does proper job within 2.5h on full data set. Needs code cleanup and qality assurance. --- .../deduplication/DeduplicateDocuments.scala | 700 ++++++++---------- .../ExecuteOnceOnExecutorsHelpers.java | 40 - .../document/deduplication/TileTask.java | 11 +- .../deduplication/TileTaskNGTest.java | 3 +- 4 files changed, 304 insertions(+), 450 deletions(-) delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index cb696062..bedc0857 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -4,13 +4,7 @@ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import java.util.function.BiPredicate import org.apache.hadoop.io.BytesWritable -import org.apache.log4j.ConsoleAppender -import org.apache.log4j.Level -import org.apache.log4j.Logger -import org.apache.log4j.PatternLayout -import org.apache.spark.HashPartitioner import org.apache.spark.SparkConf -import org.apache.spark.api.java.function.VoidFunction import pl.edu.icm.coansys.deduplication.document.voter.AuthorsVoter import pl.edu.icm.coansys.deduplication.document.voter.DoiVoter import pl.edu.icm.coansys.deduplication.document.voter.IssueVolumeVoter @@ -28,422 +22,320 @@ import org.apache.spark.storage.StorageLevel import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer -import java.io.File -import java.io.PrintWriter import pl.edu.icm.coansys.document.deduplication._ import scala.collection.JavaConverters._ object DeduplicateDocuments { - val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 7 - val maximumClusteringKeySize = 18 - val maximumClusterSize = 600 - val tileSize = 30 - - - def isValidDocument(doc: DocumentWrapper): Boolean = { - var res = false - if (doc.hasDocumentMetadata) { - val md = doc.getDocumentMetadata - if (md.hasBasicMetadata) { - val bmd = md.getBasicMetadata - if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { - res = true - } - } - } - res - } - - def calculateKey(doc: DocumentMetadata, size: Int): String = { - new CustomOddsCharsKeyGenerator(size).generateKey(doc) - } - - def calculateKeys(doc: DocumentMetadata): Array[String] = { - val keySizes = initialClusteringKeySize to maximumClusteringKeySize - val generator = new CustomOddsCharsKeyGenerator() - generator.setKeySizes(keySizes.toArray) - var res = generator.generateKeyList(doc) - //special case: if doc - if (res(0) == "") { - res = Array.fill[String](keySizes.length)(doc.getKey) - } - res - } - - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Initializing cluster preparation (V2)") - val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray - log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved - val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) - - val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) - - //build rdd (docId, (clusterId, clusterSize) ) - val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) - //build rdd - (docId, clusterId) - val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { - if (x._2 <= maximumClusterSize) { - if (y._2 <= maximumClusterSize) { - if (x._1.length <= y._1.length) { x } else { y } - } else { - x - } - } else { - if (y._2 <= maximumClusterSize) { - y - } else { - if (x._1.length > y._1.length) { x } else { y } - } - } - }).mapValues(_._1) - inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey - } - - def buildDocumentsMerger(): DuplicatesMerger = { - val res = new AdvancedDuplicatesMerger - res.setup("") - res - } - - def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { - val merger = buildDocumentsMerger() - val merged = merger.merge(docs); - merged - } - - def buildWorkComparator(): WorkComparator = { - val result = new VotesProductComparator; - result.setMinVotersWeightRequired(1.5f) - result.setProbabilityTreshold(0.5f) - result.setTresholdIncreasingVotersRequired(0.7f) - - val voters = new ListBuffer[SimilarityVoter]() - val dv = new DoiVoter() - dv.setWeight(1.0f) - voters += dv - val jv = new JournalVoter() - jv.setWeight(0.3f) - jv.setDisapproveLevel(0.5f) - jv.setApproveLevel(0.05f) - voters += jv - - val wivv = new IssueVolumeVoter - wivv.setWeight(0.3f) - wivv.setAbstainIfAbsent(true) - wivv.setSubsetResult(0.8f) - wivv.setPartiallyMatchResult(0.52f) - voters += wivv - - val wpv = new PagesVoter - wpv.setWeight(.3f) - wpv.setAbstainIfAbsent(true) - wpv.setAbsentResult(0.6f) - wpv.setSubsetResult(0.75f) - wpv.setPartiallyMatchResult(0.64f) - wpv.setRemoveRepeated(true) - voters += wpv - - val wyv = new YearVoter - wyv.setWeight(.3f) - wyv.setAbstainIfAbsent(true) - wyv.setAbsentResult(.52f) - wyv.setSubsetResult(.9f) - wyv.setPartiallyMatchResult(.75f) - wyv.setRemoveRepeated(true) - voters += wyv - - val wtv = new TitleVoter() - wtv.setWeight(0.8f) - wtv.setDisapproveLevel(0.11f) - wtv.setApproveLevel(0.001f) - wtv.setMaxNormalizedTitleLength(90) - voters += wtv - - val wav = new AuthorsVoter - wav.setWeight(0.8f) - wav.setDisapproveLevel(0.2f) - wav.setApproveLevel(0.03f) - voters += wav - - result.setSimilarityVoters(voters) - result; - } - - //size, num, min time, max time, avg time, avg_group_count - def timingStats(size: Int, measurements: Iterable[(Int, Long)]): (Int, Int, Double, Double, Double, Double) = { - val count = measurements.size - val times = measurements.map(_._2) - val minTime = times.min / 1000.0 - val maxTime = times.max / 1000.0 - val avgTime = times.sum / (1000.0 * count) - val avgGroupSize = measurements.map(_._1).sum / (1.0 * count) - (size, count, minTime, maxTime, avgTime, avgGroupSize); - } - - def clusterAnalysis(cluster: Iterable[DocumentWrapper], clusterId: String): (List[DocumentWrapper], Array[Array[Int]]) = { - log.info("Analysing cluster {}, size: {}", clusterId, cluster.size) - - val res = Array.ofDim[Int](cluster.size, cluster.size) - val sorted = cluster.toList.sortBy(dw => dw.getDocumentMetadata.getKey) - val comparator = buildWorkComparator() - - val sind = sorted.toList.zipWithIndex - - val procStart = System.currentTimeMillis - sind.foreach(p1 => { - val i1 = p1._2 - val d1 = p1._1 - //make sure diagonal is zeroed - (0 to cluster.size - 1).foreach(i => res(i)(i) = 0) - //simple iteration over table - sind.foreach(p2 => { - val i2 = p2._2 - val d2 = p2._1 - if (i1 < i2) { - val s = System.currentTimeMillis - comparator.isDuplicate(d1.getDocumentMetadata, d2.getDocumentMetadata, null) - val e = System.currentTimeMillis - res(i1)(i2) = (e - s).toInt - } - }) - }) - val procEnd = System.currentTimeMillis - val elapsedSteps = res.map(_.sum).sum - log.info("Finished processing, elapsed: %.3f, sum of steps: %.3f".format((procEnd - procStart) / 1000.0, elapsedSteps / 1000.0)) - - (sorted, res) - } - - //todo: timing depending on size - //Deduplicates the documents within single cluster, and creates a number of clusters containing documensts which - //are assumed to be duplicates. - def deduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): Iterable[(String, List[DocumentWrapper])] = { - log.info("Deduplicating cluster {}, size: {}", clusterId, cluster.size) - // - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - if (cluster.size > 30) { - - log.info("Sample document: \n{}", cluster.head.getDocumentMetadata.getBasicMetadata) - } - val startTime = System.currentTimeMillis(); - val classes = ListBuffer[ListBuffer[DocumentWrapper]]() - val comparator = buildWorkComparator() - cluster.foreach(x => { - val arr = classes.find(a => { comparator.isDuplicate(x.getDocumentMetadata, a(0).getDocumentMetadata, null) }) - if (arr.isEmpty) { - val nclass = ListBuffer(x) - classes += nclass - } else { - arr.get += x - } - }) - val t: Double = (System.currentTimeMillis() - startTime) / 1000.0 - val res = classes.zipWithIndex.map(x => (clusterId + x._2, x._1.toList)) - log.info("Deduplicated cluster " + clusterId + ", size " + cluster.size + " in " + t + " sec into " + res.size + " groups.") - - res + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) + val initialClusteringKeySize = 7 + val maximumClusteringKeySize = 14 + val maximumClusterSize = 2000 + val tileSize = 100 + + + implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = + new BiPredicate[A, B] { + def test(a: A, b: B) = predicate(a, b) } - def timedDeduplicateCluster(cluster: Iterable[DocumentWrapper], clusterId: String): (Iterable[(String, List[DocumentWrapper])], (Int, (Int, Long))) = { - val start = System.currentTimeMillis; - val rresult = deduplicateCluster(cluster, clusterId); - val end = System.currentTimeMillis - - (rresult, (cluster.size, (rresult.size, end - start))) - } - implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = - new BiPredicate[A, B] { - def test(a: A, b: B) = predicate(a, b) + def isValidDocument(doc: DocumentWrapper): Boolean = { //todo: fix based on if return value. + var res = false; + if (doc.hasDocumentMetadata()) { + val md = doc.getDocumentMetadata + if (md.hasBasicMetadata) { + val bmd = md.getBasicMetadata + if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { + res = true } - -// -// implicit def toVoidFunction[T](runMethod: (T) => Unit) = -// new VoidFunction[T]() { -// def call(t:T) = runMethod(t) -// } - - - def reconfigureLogging(sc: SparkContext) = { - ExecuteOnceOnExecutorsHelpers.executeEverywhere(sc, "RECONFIGURE_LOGGING", true, - new VoidFunction[java.util.Iterator[_]]() { - def call(t :java.util.Iterator[_]) :Unit={ - Logger.getLogger("pl.edu.icm.coansys.deduplication.document.comparator.AbstractWorkComparator").setLevel(Level.WARN) - val console = new ConsoleAppender(new PatternLayout("%d [%p|%c|%C{1}] %m%n"), ConsoleAppender.SYSTEM_OUT); //create appender - console.activateOptions(); - Logger.getLogger("pl.edu.icm").addAppender(console) - log.info("Reconfigured logger...") - } - } - ) + } } + res + } - /** - * @param args the command line arguments - */ - def main(args: Array[String]): Unit = { - val enableClusterSummary = false; - val fixInvalidDocuments = true; - val removeDoubles = true; - - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supported, exitting.") - return ; - } else { - println("Arguments:") - args.foreach(println); - } - println("Creating context...") - - val conf = new SparkConf() - .setAppName("Document deduplication") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") - val sc = new SparkContext(conf) - reconfigureLogging(sc) - - println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" - val outputDocuments = args(1) - - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - log.info("Loaded raw bytes.") - - val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) - - //fix invalid documents: - val fixedWrappers = if (fixInvalidDocuments) { - val x = dirtyWrappers.filter(w => isValidDocument(w._2)) - val afterSize = x.count; - val preSize = dirtyWrappers.count - log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") - x + def calculateKey(doc: DocumentMetadata, size: Int): String = { + new CustomOddsCharsKeyGenerator(size).generateKey(doc) + } + + def calculateKeys(doc: DocumentMetadata): Array[String] = { + val keySizes = initialClusteringKeySize to maximumClusteringKeySize + val generator = new CustomOddsCharsKeyGenerator() + generator.setKeySizes(keySizes.toArray) + var res = generator.generateKeyList(doc) + //special case: if doc has no title. + if (res(0) == "") { + res = Array.fill[String](keySizes.length)(doc.getKey) + } + res + } + + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + log.info("Initializing cluster preparation (V2)") + val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Will use key sizes: " + keySizes.mkString(", ")) + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) + + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + //build rdd (docId, (clusterId, clusterSize) ) + val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) + //build rdd - (docId, clusterId) + val selectedClusters = docClustersWithSizes.reduceByKey((x, y) => { + if (x._2 <= maximumClusterSize) { + if (y._2 <= maximumClusterSize) { + if (x._1.length <= y._1.length) { x } else { y } } else { - dirtyWrappers + x } - - val wrappers = if (removeDoubles) { - fixedWrappers.reduceByKey((x, y) => y) + } else { + if (y._2 <= maximumClusterSize) { + y } else { - fixedWrappers + if (x._1.length > y._1.length) { x } else { y } } - wrappers.persist(StorageLevel.MEMORY_AND_DISK) - val initialSize = wrappers.count - log.info(f"Starting processing with $initialSize documents.") - - val initialGroups = prepareClustersV2(wrappers) - initialGroups.persist(StorageLevel.MEMORY_AND_DISK) - + } + }).mapValues(_._1) + inputDocs.join(selectedClusters).map(p => (p._2._2, p._2._1)).groupByKey + } + + def buildDocumentsMerger(): DuplicatesMerger = { + val res = new AdvancedDuplicatesMerger + res.setup("") + res + } + + def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { + val merger = buildDocumentsMerger() + val merged = merger.merge(docs); + merged + } + + def buildWorkComparator(): WorkComparator = { + val result = new VotesProductComparator; + result.setMinVotersWeightRequired(1.5f) + result.setProbabilityTreshold(0.5f) + result.setTresholdIncreasingVotersRequired(0.7f) + + val voters = new ListBuffer[SimilarityVoter]() + val dv = new DoiVoter() + dv.setWeight(1.0f) + voters += dv + val jv = new JournalVoter() + jv.setWeight(0.3f) + jv.setDisapproveLevel(0.5f) + jv.setApproveLevel(0.05f) + voters += jv + + val wivv = new IssueVolumeVoter + wivv.setWeight(0.3f) + wivv.setAbstainIfAbsent(true) + wivv.setSubsetResult(0.8f) + wivv.setPartiallyMatchResult(0.52f) + voters += wivv + + val wpv = new PagesVoter + wpv.setWeight(.3f) + wpv.setAbstainIfAbsent(true) + wpv.setAbsentResult(0.6f) + wpv.setSubsetResult(0.75f) + wpv.setPartiallyMatchResult(0.64f) + wpv.setRemoveRepeated(true) + voters += wpv + + val wyv = new YearVoter + wyv.setWeight(.3f) + wyv.setAbstainIfAbsent(true) + wyv.setAbsentResult(.52f) + wyv.setSubsetResult(.9f) + wyv.setPartiallyMatchResult(.75f) + wyv.setRemoveRepeated(true) + voters += wyv + + val wtv = new TitleVoter() + wtv.setWeight(0.8f) + wtv.setDisapproveLevel(0.11f) + wtv.setApproveLevel(0.001f) + wtv.setMaxNormalizedTitleLength(90) + voters += wtv + + val wav = new AuthorsVoter + wav.setWeight(0.8f) + wav.setDisapproveLevel(0.2f) + wav.setApproveLevel(0.03f) + voters += wav + + result.setSimilarityVoters(voters) + result; + } + + + /** + * @param args the command line arguments + */ + def main(args: Array[String]): Unit = { + val enableClusterSummary = false; + val fixInvalidDocuments = true; + val removeDoubles = true; + + println("Starting app") + //load the file: + if (args.size == 0) { + println("No args supplied, exitting.") + return ; + } else { //todo: add arguments interpretation. + println("Arguments:") + args.foreach(println); + } + println("Creating context...") + + val conf = new SparkConf() + .setAppName("Document deduplication") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "pl.edu.icm.coansys.document.deduplication.DocumentWrapperKryoRegistrator") + + val sc = new SparkContext(conf) + + println("Created context...") + sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = args(0) + // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + // "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" + val outputDocuments = args(1) + + val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) + println("Loaded raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if (fixInvalidDocuments) { + val x = dirtyWrappers.filter(w => isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") + x + } else { + dirtyWrappers + } - if (enableClusterSummary) { - //prepare cluster summary - val largeClusters = initialGroups.filter(_._2.size > 1000) - largeClusters.mapValues(_.size).take(250).foreach(println) - val sampledResult = largeClusters.mapValues(x => (x.size, x.take(10))).collect - // val result = largeClusters.collect() - sampledResult.foreach(x => { - val docs = x._2._2 - val clusterId = x._1 - val fileBase = clusterId + "-%04d-".format(docs.size) - println(fileBase) - - println("Writing docs...") - var pw = new PrintWriter(new File(fileBase + "docs.txt"), "UTF-8") - docs.foreach(x => { pw.println(x.getDocumentMetadata()); pw.println("====================") }) - pw.close - }) - return + val wrappers = if (removeDoubles) { + fixedWrappers.reduceByKey((x, y) => y) + } else { + fixedWrappers + } +// wrappers.persist(StorageLevel.MEMORY_AND_DISK) + + val initialSize = wrappers.count + println(f"Starting processing with $initialSize documents.") + + val initialGroups = prepareClustersV2(wrappers) + initialGroups.persist//(StorageLevel.MEMORY_AND_DISK) + + val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) + val initialClusterCount = clustersToDeduplicate.count + //TODO: some statistics here on cluster, would be useful. + + val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) + tiledTasks.persist//(StorageLevel.MEMORY_AND_DISK) + val tileCount = tiledTasks.count; + + println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") + // val comparator = sc.broadcast(buildWorkComparator) //todo: can we do comparator broadcast variable? + + val partialEqualityClusters = tiledTasks.flatMap( + v => { + val t0 = java.lang.System.currentTimeMillis() + // log.info("Starting tile task %s".format(v.getTaskId)) + val comparator = buildWorkComparator + val res = v.processPairs((a: DocumentWrapper, b: DocumentWrapper) => + comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) + val time = (java.lang.System.currentTimeMillis() - t0) / 1000.0 + log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") + res + } + ) + // At this moment we have RDD of (String, Seq[String]) (clusterId and list of 'equal' ids inside this cluster + // let's save it: + // partialEqualityClusters.saveAsObjectFile ("hdfs:///user/axnow/intermediate/pec"); + // //later, in other program - watch, you have tuples out of the box :) + // val loadedPec = sc.objectFile[(String, java.util.List[String])]("intermediate/pec") + // val partialEqualityClusters = loadedPec + // + // + // equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) + // log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); + // val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) + // log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); + // + // val eqclsizes= filteredSizes.collect.sortBy (_._2) + // println("Equality cluster sizes:") + // eqclsizes.foreach(println(_)) + // println("Done.\n\n") + + val finalClusters = partialEqualityClusters.mapValues(List(_)). + reduceByKey(_ ++ _). //one long list of lists of ids + map(pair => { + val t0 = java.lang.System.currentTimeMillis() + val res = TileTask.coalesceResult(pair._2.asJava) + val tt = System.currentTimeMillis() - t0; + val clusterSize=pair._2.size + log.info(f"Finished tile coalesce task. (Cluster,time[s], size): ${pair._1}, ${tt / 1000.0}, ${pair._2.size}") + (pair._1, res) + }). + flatMap( //build proper ids for equality clusters. + p => { + val cid = p._1 + val cl = p._2 + cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) } - - // val igs = initialGroups.count() - val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) - val initialClusterCount = clustersToDeduplicate.count - - val tiledTasks = clustersToDeduplicate.flatMap( p => TileTask.parallelize(p._1, p._2, tileSize) ) - val shuffledTiledTasks = tiledTasks.repartition(5000); - shuffledTiledTasks.persist(StorageLevel.MEMORY_AND_DISK) - val tileCount = shuffledTiledTasks.count; - - log.info(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") -// val comparator = sc.broadcast(buildWorkComparator) - reconfigureLogging(sc) - val partialEqualityClusters = shuffledTiledTasks.flatMap( - v => { - val t0=java.lang.System.currentTimeMillis() - log.info("Starting tile task %s".format(v.getTaskId)) - val comparator = buildWorkComparator - val res = v.processPairs( (a: DocumentWrapper, b: DocumentWrapper) => - comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) - val time = (java.lang.System.currentTimeMillis()-t0)/1000. - log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") - res - } - ) -// equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) -// log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); -// val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) -// log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); -// -// val eqclsizes= filteredSizes.collect.sortBy (_._2) -// println("Equality cluster sizes:") -// eqclsizes.foreach(println(_)) -// println("Done.\n\n") - - val finalClusters = partialEqualityClusters.mapValues(List(_)). - reduceByKey( - (a, b) => TileTask.coalesceResult(a, b).toList - ).flatMap( - p => { - val cid = p._1 - val cl = p._2 - cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) - }) - - finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); - val finclSizes=finalClusters.mapValues(_.size).filter(_._2>500).collect.sortBy(_._1) - println("\n\nFinal cluster sizes:") - finclSizes.foreach(println(_)) - println("Done.") - - val finalDocClusters = finalClusters.flatMapValues(x => x). - map(v => (v._2, v._1)).join(wrappers).map(v => (v._2._1, v._2._2)). - groupByKey() + ) + //now we got all the items in place + finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); + val finclSizes = finalClusters.mapValues(_.size).takeOrdered(100)(Ordering[Int].on(-_._2)) + println("Top 100 cluster sizes:") + finclSizes.foreach(println(_)) + println("-----\n\n") + + //count clusters, documents in clusters and number of documents to be deduced: + val finalClusterCount = finalClusters.count + val documentInFinalClusterCount = finalClusters.map(_._2.size).reduce(_ + _) + val documentRemovedDuringClusteringCount = documentInFinalClusterCount - finalClusterCount + println(f"Finally created $finalClusterCount clusters, containing $documentInFinalClusterCount documents, $documentRemovedDuringClusteringCount documents will be removed.") + + // merge documents + val docIdWithClusterId = finalClusters.flatMapValues(x => x). + map(v => (v._2, v._1)) + val documentWrappersPrepared = wrappers.leftOuterJoin(docIdWithClusterId); + val mergedDocuments = documentWrappersPrepared.filter(_._2._2.isDefined). + map(x => (x._2._2, List(x._2._1))).reduceByKey(_ ++ _). //get lists of cluster documents by cluster id + map(kv => { + val doc = mergeDocuments(kv._2) + (doc.getDocumentMetadata.getKey, doc) + }) + + // documents not touched + val singularDocuments = documentWrappersPrepared.filter(_._2._2.isEmpty).map(x => (x._1, x._2._1)) + + //final result. + val finalResult = singularDocuments.union(mergedDocuments) + finalResult.persist - val singularClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size==1) - val pluralClustersAfterDeduplication = finalDocClusters.filter(p=>p._2.size>1) - - log.info("After reducing clusters (comparison)") - val merged = pluralClustersAfterDeduplication.map(x => (mergeDocuments(x._2.toList))) - log.info("Finished merge") - // val mergedSize = merged.count() - - val single = initialGroups.filter(t => t._2.size == 1) - // log.info("Got initial group count=" + igs + "; singular=" + single.count + - // "; multiple=" + clustersToDeduplicate.count + "; reducedCount=" + deduplicatedClusters.count()) - // - //now merge the arrays: - val toWrite = merged.union(single.union(singularClustersAfterDeduplication).map(x => x._2.head)).map(doc => (doc.getDocumentMetadata.getKey, doc)) - if ("-" != outputDocuments) { - val bas = toWrite.mapValues(doc => doc.toByteArray()) - bas.saveAsSequenceFile(outputDocuments); - } else { - log.info("Simulating timing by counting.") - toWrite.count() - } -// val tgrouped = timing.groupByKey; -// val stats = tgrouped.map(x => timingStats(x._1, x._2)).collect -// println("================ Timing stats ======================") -// stats.sortBy(x => x._1).map(_.productIterator.toList.mkString(",")).foreach(println) -// println("================ end of timing stats ======================") - - println("Exit") - + val finalSize = finalResult.count + println(f"Final counts:\n-----------\n"+ + f" input: $initialSize\n"+ + f" output: $finalSize\n"+ + f" removed: $documentRemovedDuringClusteringCount\n"+ + f" difference: ${initialSize-finalSize-documentRemovedDuringClusteringCount}") + + if ("-" != outputDocuments) { + val bas = finalResult.mapValues(doc => doc.toByteArray()).saveAsSequenceFile(outputDocuments); + } else { + log.info("Simulating timing by counting.") + finalResult.count() + println("Finished counting.") } + } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java deleted file mode 100644 index a413ad20..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/ExecuteOnceOnExecutorsHelpers.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; -import org.apache.spark.SparkContext; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.VoidFunction; - -/** - * - * @author Aleksander Nowinski - */ -public class ExecuteOnceOnExecutorsHelpers { - - private static Set executedCode = new HashSet<>(); - - public static void executeEverywhere(SparkContext sc, String codeId, boolean runOnDriver, VoidFunction function) throws Exception{ - JavaSparkContext.fromSparkContext(sc).parallelize(Arrays.asList(new Integer[1000])).foreachPartition(new VoidFunction>() { - @Override - public void call(Iterator it) throws Exception { - if (!executedCode.contains(codeId)) { - ExecuteOnceOnExecutorsHelpers.executedCode.add(codeId); - function.call(it); - } - } - }); - if (runOnDriver) { - function.call(null); - } - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java index 4af0524e..84130424 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java @@ -91,16 +91,17 @@ public boolean equals(Object obj) { public List>> processPairs(BiPredicate equalityTest) { List> raw = new ArrayList<>(); for (DocumentWrapper row : rows) { - final List rlist = new ArrayList<>(); - rlist.add(row.getDocumentMetadata().getKey()); List equalColums = columns.stream() .filter( column -> (row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey()) < 0) && (equalityTest.test(row, column)) ).map(column -> column.getDocumentMetadata().getKey()) .collect(Collectors.toList()); - rlist.addAll(equalColums); - raw.add(rlist); + if (!equalColums.isEmpty()) { + final List rlist = new ArrayList<>(equalColums); + rlist.add(row.getDocumentMetadata().getKey()); + raw.add(rlist); + } } raw = coalesceResult(raw); return raw.stream().map(list -> new Tuple2<>(clusterId, list)).collect(Collectors.toList()); @@ -158,7 +159,7 @@ public static List> coalesceResult(List> clusters) { List> res = new ArrayList<>(); while (!all.isEmpty()) { Set current = all.remove(0); - boolean anyChange = false; + boolean anyChange; do { anyChange = false; ListIterator> li = all.listIterator(); diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java index 9b3ebd49..1b5ad200 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java @@ -57,7 +57,8 @@ protected List createDocumentList(int size) { /** * Test of processPairs method, of class TileTask. */ - @Test +// @Test +// @Ignore public void testProcessPairs() { System.out.println("processPairs"); int size = 500; From ac56042954d7919e3cfff48e739031ec70aacf67 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Mon, 26 Jun 2017 09:38:59 +0200 Subject: [PATCH 17/20] Added options parsing from command line to control app behaviour. Version used for performance testing. --- .../deduplication/DeduplicateDocuments.scala | 114 ++++++++++++------ 1 file changed, 77 insertions(+), 37 deletions(-) diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index bedc0857..8211560b 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -27,19 +27,12 @@ import scala.collection.JavaConverters._ object DeduplicateDocuments { val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) - val initialClusteringKeySize = 7 - val maximumClusteringKeySize = 14 - val maximumClusterSize = 2000 - val tileSize = 100 - - implicit def toJavaBiPredicate[A, B](predicate: (A, B) => Boolean) = new BiPredicate[A, B] { def test(a: A, b: B) = predicate(a, b) } - def isValidDocument(doc: DocumentWrapper): Boolean = { //todo: fix based on if return value. var res = false; if (doc.hasDocumentMetadata()) { @@ -53,13 +46,12 @@ object DeduplicateDocuments { } res } - def calculateKey(doc: DocumentMetadata, size: Int): String = { new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - def calculateKeys(doc: DocumentMetadata): Array[String] = { + def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Array[String] = { val keySizes = initialClusteringKeySize to maximumClusteringKeySize val generator = new CustomOddsCharsKeyGenerator() generator.setKeySizes(keySizes.toArray) @@ -71,11 +63,12 @@ object DeduplicateDocuments { res } - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)]): RDD[(String, Iterable[DocumentWrapper])] = { + def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)], initialClusteringKeySize: Int, + maximumClusteringKeySize: Int, maximumClusterSize: Int): RDD[(String, Iterable[DocumentWrapper])] = { log.info("Initializing cluster preparation (V2)") val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata())); //we loose documents here, ony ids are preseved + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)); //we loose documents here, ony ids are preseved val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) @@ -171,24 +164,67 @@ object DeduplicateDocuments { result; } - + case class Config( + inputFile: String = "", + outputFile: String = "", + dumpClusters: Boolean = false, + keySizeMin: Int = 5, + keySizeMax: Int = 10, + clusterSizeMax: Int = 500, + tileSize: Int = 25 + ) /** * @param args the command line arguments */ def main(args: Array[String]): Unit = { - val enableClusterSummary = false; val fixInvalidDocuments = true; val removeDoubles = true; - println("Starting app") - //load the file: - if (args.size == 0) { - println("No args supplied, exitting.") - return ; - } else { //todo: add arguments interpretation. - println("Arguments:") - args.foreach(println); + println("Starting document deduplication") + + val parser = new scopt.OptionParser[Config]("CoAnSys Deduplicate Documents") { + head("Deduplicate documents", "0.1") + +// opt[Unit]('c', "dump-clusters").action((x, c) => +// c.copy(dumpClusters = true)).text("dump similarity clusters into debug/clusters hdfs dir during run") + + opt[Int]("cluster-key-min").abbr("kmn").action((x, c) => c.copy(keySizeMin = x)). + validate(x => + if (x >= 2) success + else failure("Value must be >=2")). + text("shortest valid key for cluster, defines pre-clustering. Recommended value more thab 4, minimum 2.") + + opt[Int]("cluster-key-max").abbr("kmx").action((x, c) => c.copy(keySizeMax = x)). + validate(x => + if (x >= 2 && x <= 20) success + else failure("Value must be >=2")). + text("longest valid key for cluster, during pre-clustering. Used to split large clusters. Recommended value more than min, minimum 2, max 20.") + + opt[Int]("cluster-size-max").abbr("cs").action((x, c) => c.copy(clusterSizeMax = x)). + text("Largest acceptable cluster size during preclustering phase. If cluster exceeds algorithm attempts to use longer key if possible. Typically 400+") + + opt[Int]("tile-size").abbr("ts").action((x, c) => c.copy(keySizeMax = x)). + validate(x => + if (x >= 2) success + else failure("Value must be >=2")). + text("Size of the tile tasks used to split large clusters. Min 2, recommended approx 40") + + arg[String]("").required.text("Input sequence file").action((f, c) => c.copy(inputFile = f)) + arg[String]("").optional.text("Output sequence file. If ommited, then no output is written but calculation is done.").action((f, c) => c.copy(outputFile = f)) + note("Blah") + } + + val cfg: Config = parser.parse(args, Config()) match { + case Some(config) => + println(f"Got config:\n${config}") + println(config); + config + case None => + // arguments are bad, error message will have been displayed + println("No config.") + return } + println("Creating context...") val conf = new SparkConf() @@ -199,11 +235,15 @@ object DeduplicateDocuments { val sc = new SparkContext(conf) println("Created context...") - sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) - val inputDocuments = args(0) + // sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) + val inputDocuments = cfg.inputFile // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" // "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" - val outputDocuments = args(1) + val outputDocuments = cfg.outputFile + val tileSize = cfg.tileSize + val initialClusteringKeySize = cfg.keySizeMin + val maximumClusteringKeySize = cfg.keySizeMax + val maximumClusterSize = cfg.clusterSizeMax val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) println("Loaded raw bytes.") @@ -226,20 +266,20 @@ object DeduplicateDocuments { } else { fixedWrappers } -// wrappers.persist(StorageLevel.MEMORY_AND_DISK) + // wrappers.persist(StorageLevel.MEMORY_AND_DISK) val initialSize = wrappers.count println(f"Starting processing with $initialSize documents.") - val initialGroups = prepareClustersV2(wrappers) - initialGroups.persist//(StorageLevel.MEMORY_AND_DISK) + val initialGroups = prepareClustersV2(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) + initialGroups.persist //(StorageLevel.MEMORY_AND_DISK) val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) val initialClusterCount = clustersToDeduplicate.count //TODO: some statistics here on cluster, would be useful. val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) - tiledTasks.persist//(StorageLevel.MEMORY_AND_DISK) + tiledTasks.persist //(StorageLevel.MEMORY_AND_DISK) val tileCount = tiledTasks.count; println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") @@ -281,7 +321,7 @@ object DeduplicateDocuments { val t0 = java.lang.System.currentTimeMillis() val res = TileTask.coalesceResult(pair._2.asJava) val tt = System.currentTimeMillis() - t0; - val clusterSize=pair._2.size + val clusterSize = pair._2.size log.info(f"Finished tile coalesce task. (Cluster,time[s], size): ${pair._1}, ${tt / 1000.0}, ${pair._2.size}") (pair._1, res) }). @@ -322,15 +362,15 @@ object DeduplicateDocuments { //final result. val finalResult = singularDocuments.union(mergedDocuments) finalResult.persist - + val finalSize = finalResult.count - println(f"Final counts:\n-----------\n"+ - f" input: $initialSize\n"+ - f" output: $finalSize\n"+ - f" removed: $documentRemovedDuringClusteringCount\n"+ - f" difference: ${initialSize-finalSize-documentRemovedDuringClusteringCount}") - - if ("-" != outputDocuments) { + println(f"Final counts:\n-----------\n" + + f" input: $initialSize\n" + + f" output: $finalSize\n" + + f" removed: $documentRemovedDuringClusteringCount\n" + + f" difference: ${initialSize - finalSize - documentRemovedDuringClusteringCount}") + + if ("-" != outputDocuments && !outputDocuments.isEmpty) { val bas = finalResult.mapValues(doc => doc.toByteArray()).saveAsSequenceFile(outputDocuments); } else { log.info("Simulating timing by counting.") From 81f6509ea01785c54fe73a31264a871a29413852 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Tue, 11 Jul 2017 14:40:40 +0200 Subject: [PATCH 18/20] Added dependency for the scopt. Task tiling class rewritten to scala, with tests. --- .../deduplication-document-spark-impl/pom.xml | 32 ++- .../deduplication/CartesianTaskSplit.scala | 87 ++++++++ .../CustomOddsCharsKeyGenerator.java | 136 ------------ .../deduplication/DeduplicateDocuments.scala | 91 ++++---- .../DocumentWrapperKryoRegistrator.java | 10 +- .../MultiLengthTitleKeyGenerator.scala | 54 +++++ .../document/deduplication/TileTask.java | 200 ------------------ .../CartesianTaskSplitSuite.scala | 88 ++++++++ .../CustomOddsCharsKeyGeneratorTest.java | 110 ---------- .../MultiLengthTitleKeyGeneratorSuite.scala | 67 ++++++ .../deduplication/TileTaskNGTest.java | 165 --------------- 11 files changed, 367 insertions(+), 673 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala delete mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index 6d6263c9..aa9a9145 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -109,9 +109,24 @@ scp://cypisek-gw.ocean.icm.edu.pl/home/axnow/jobs/ dedupdocs.jar - - - + + + org.scalatest + scalatest-maven-plugin + 1.0 + + ${project.build.directory}/surefire-reports + . + WDF TestSuite.txt + + + + test + + test + + + @@ -155,5 +170,16 @@ guava 15.0 + + com.github.scopt + scopt_2.10 + 3.6.0 + + + org.scalatest + scalatest_2.10 + 3.0.1 + test + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala new file mode 100644 index 00000000..2bc76c42 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala @@ -0,0 +1,87 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper + +class CartesianTaskSplit( + val clusterId: String, + val taskId: String, + val rows: Seq[DocumentWrapper], + val columns: Seq[DocumentWrapper] +) { + /** + * Generate list of clusters of the documents, where predicate is conformed, ie + * function passed returned true. The predicate is assumed to be + * symmetrical, so it is executed only once on each pair. Note, that as we + * expect that all the tiles will appear within the task, and the comparison + * operator may be expensive, only situations where row key is lesser than + * column key are taken into account + * + * @param equalityTest predicate which defines whether or no two elements + * are considered matching (typically equal) + * @return list of lists of keys of equal documents (documents where + * equalityTest returned true) + */ + def processPairs(equalityTest: (DocumentWrapper, DocumentWrapper) => Boolean): Seq[Seq[String]] = { + return List.empty + + val clusters: Seq[Seq[String]] = rows.map(row => { + val rkey = row.getDocumentMetadata.getKey + val equalColumnKeys = columns.filter(rkey < _.getDocumentMetadata.getKey) + .filter(equalityTest(row, _)) + .map(_.getDocumentMetadata.getKey) + equalColumnKeys :+ rkey + }).filter(_.size > 1) + CartesianTaskSplit.coalesceClusters(clusters) + } + +} + +object CartesianTaskSplit { + /** + * Combine clusters which have non-empty intersection, so result will be + * only separate lists. + * + * @param clusters lists to combine + * @return list of the separate clusters, obtained from merging input clusters + */ + def coalesceClusters(clusters: Seq[Seq[String]]): Seq[Seq[String]] = { + var sets = clusters.map(_.toSet[String]) + var res = List.empty[Set[String]] + while (!sets.isEmpty) { + var current = sets.head + sets = sets.tail + var ps: (Seq[Set[String]], Seq[Set[String]]) = null + do { + ps = sets.partition(_.exists(current.contains(_))) + current +: ps._1.flatMap(x => x) + sets = ps._2 + } while (!ps._1.isEmpty) + res :+ current + } + res.map(_.toSeq) + } + + /** Split one large cluster into parallel tasks of the given size. + */ + def parallelizeCluster(clusterId: String, documents: Iterable[DocumentWrapper], tileSize: Int): Seq[CartesianTaskSplit] = { + println(f"Document count: ${documents.size}, tile size $tileSize") + val ntiles = documents.size/tileSize + (if(documents.size % tileSize>0) 1 else 0) + println(f"ntiles: $ntiles") + + val sdoc = documents.toSeq.sorted(Ordering.by[DocumentWrapper, String](_.getDocumentMetadata.getKey)) + val groupedDocs = sdoc.zipWithIndex.map(docidx => (docidx._2%ntiles, docidx._1)).groupBy[Int](_._1).mapValues(_.map(_._2)) + val res = groupedDocs.flatMap(kv => + groupedDocs.map(kvin => new CartesianTaskSplit( + clusterId, f"${clusterId}_${kv._1}:${kv._2}",kv._2, kvin._2 + ) + ) + ) + res.toSeq + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java deleted file mode 100644 index 442208ff..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGenerator.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * This file is part of CoAnSys project. - * Copyright (c) 2012-2015 ICM-UW - * - * CoAnSys is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - - * CoAnSys is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with CoAnSys. If not, see . - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.util.Arrays; -import pl.edu.icm.coansys.commons.java.DocumentWrapperUtils; -import pl.edu.icm.coansys.commons.java.StringTools; -import pl.edu.icm.coansys.deduplication.document.keygenerator.WorkKeyGenerator; -import pl.edu.icm.coansys.models.DocumentProtos; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; - -/** - * An early deduplication phase key used to group works into clusters. This is - * an extended version, which allows to define requested key size - * - * @author Łukasz Dumiszewski - * @author Aleksander Nowiński - * - */ -public class CustomOddsCharsKeyGenerator implements WorkKeyGenerator { - - public static final int DEFAULT_KEY_SIZE = 5; - - private int keySize = DEFAULT_KEY_SIZE; - - private int[] keySizes = {DEFAULT_KEY_SIZE}; - - public CustomOddsCharsKeyGenerator() { - } - - public CustomOddsCharsKeyGenerator(int keySize) { - this.keySize = keySize; - } - - /** - * Generates reasonable deduplication key for the given - * {@link DocumentWrapper}. The key has size as defined by keySize param. - * The key is created using title of the work, with removed punctuation and - * basic english stopwords, and then keySize odd letters are taken. The - * method is thread-safe. - * - * @param doc document, which key is generated for - * @return an reasonable key for starting deduplication of size keySize - */ - @Override - public String generateKey(DocumentProtos.DocumentMetadata doc) { - String title = DocumentWrapperUtils.getMainTitle(doc); - return generateKey(title); - } - - /** - * Generate a collection of keys of predefined sizes. - * - * @param doc - * @return - */ - public String[] generateKeyList(DocumentProtos.DocumentMetadata doc) { - String title = DocumentWrapperUtils.getMainTitle(doc); - return generateKeys(title); - } - - protected String[] generateKeys(String title) { - title = cleanUpString(title); - String[] res = new String[keySizes.length]; - for (int k = 0; k < keySizes.length; k++) { - int kl = keySizes[k]; - StringBuilder oddCharsSB = new StringBuilder(); - for (int i = 0; i < title.length() && oddCharsSB.length() < kl; i += 2) { - oddCharsSB.append(title.charAt(i)); - } - res[k] = oddCharsSB.toString(); - } - return res; - } - - protected String generateKey(String title) { - title = cleanUpString(title); - - StringBuilder oddCharsSB = new StringBuilder(); - for (int i = 0; i < title.length() && oddCharsSB.length() < keySize; i += 2) { - oddCharsSB.append(title.charAt(i)); - } - return oddCharsSB.toString(); - } - - protected String cleanUpString(String title) { - title = StringTools.normalize(title); //fixme: it seems that normalize, despite javadocs has stopword removal already - title = StringTools.removeStopWords(title); - title = title.replaceAll("\\s", ""); - return title; - } - - public int getKeySize() { - return keySize; - } - - public void setKeySize(int keySize) { - this.keySize = keySize; - } - - public int[] getKeySizes() { - return Arrays.copyOf(keySizes, keySizes.length); - } - - public void setKeySizes(int[] keySizes) { - if (keySizes == null) { - throw new IllegalArgumentException("Null sizes not premitted"); - } - if (keySizes.length < 1) { - throw new IllegalArgumentException("Non empty array required"); - } - for (int i = 0; i < keySizes.length - 1; i++) { - if (keySizes[i] >= keySizes[i + 1]) { - throw new IllegalArgumentException("Array must be sorted in growing order and no equal sizes present."); - } - - } - this.keySizes = Arrays.copyOf(keySizes, keySizes.length); - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 8211560b..378c8e1f 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -34,44 +34,45 @@ object DeduplicateDocuments { } def isValidDocument(doc: DocumentWrapper): Boolean = { //todo: fix based on if return value. - var res = false; if (doc.hasDocumentMetadata()) { val md = doc.getDocumentMetadata if (md.hasBasicMetadata) { val bmd = md.getBasicMetadata - if (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) { - res = true - } + (bmd.getTitleCount() > 0 || bmd.getAuthorCount > 0 || bmd.hasDoi || bmd.hasJournal) + } else { + false } + } else { + false } - res - } - - def calculateKey(doc: DocumentMetadata, size: Int): String = { - new CustomOddsCharsKeyGenerator(size).generateKey(doc) } - def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Array[String] = { + def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Seq[String] = { val keySizes = initialClusteringKeySize to maximumClusteringKeySize - val generator = new CustomOddsCharsKeyGenerator() - generator.setKeySizes(keySizes.toArray) - var res = generator.generateKeyList(doc) - //special case: if doc has no title. - if (res(0) == "") { + var res = MultiLengthTitleKeyGenerator.generateKeys(doc)(keySizes) + if (res.head.isEmpty) { res = Array.fill[String](keySizes.length)(doc.getKey) } res } - def prepareClustersV2(inputDocs: RDD[(String, DocumentWrapper)], initialClusteringKeySize: Int, + /** + * Group items into large clusters, within which detailed analysis will be + * held. Items are grouped by keys generated from the normalised titles. + * If the cluster is too big, then longer keys are used, so smaller clusters are + * generated. Treshold is maximumClusterSize. + * + */ + def prepareInitialClustering(inputDocs: RDD[(String, DocumentWrapper)], initialClusteringKeySize: Int, maximumClusteringKeySize: Int, maximumClusterSize: Int): RDD[(String, Iterable[DocumentWrapper])] = { - log.info("Initializing cluster preparation (V2)") - val keySizes = (initialClusteringKeySize to maximumClusteringKeySize).toArray + log.info("Initializing cluster preparation.") + val keySizes = initialClusteringKeySize to maximumClusteringKeySize log.info("Will use key sizes: " + keySizes.mkString(", ")) - val idClusterKeys = inputDocs.mapValues(doc => calculateKeys(doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)); //we loose documents here, ony ids are preseved - val clusterDoc = idClusterKeys.flatMap(p => p._2.map(idcluster => (idcluster, p._1))) - - val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) + + val idClusterKeys = inputDocs.mapValues(doc => calculateKeys( + doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)) //we loose documents here, ony ids are preseved + val clusterDoc = idClusterKeys.flatMap(kv => kv._2.map(idcluster => (idcluster, kv._1))) // (clusterId => docId) + val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) //(clusterId => clusterSize) //build rdd (docId, (clusterId, clusterSize) ) val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) @@ -237,8 +238,9 @@ object DeduplicateDocuments { println("Created context...") // sc.getConf.getAll.foreach(x => println(x._1 + ": " + x._2)) val inputDocuments = cfg.inputFile - // "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" - // "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" + // for pasting into console: + // val inputDocuments = "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" + // val inputDocuments = "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" val outputDocuments = cfg.outputFile val tileSize = cfg.tileSize val initialClusteringKeySize = cfg.keySizeMin @@ -266,60 +268,45 @@ object DeduplicateDocuments { } else { fixedWrappers } - // wrappers.persist(StorageLevel.MEMORY_AND_DISK) val initialSize = wrappers.count println(f"Starting processing with $initialSize documents.") - val initialGroups = prepareClustersV2(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) + val initialGroups = prepareInitialClustering(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) initialGroups.persist //(StorageLevel.MEMORY_AND_DISK) val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) val initialClusterCount = clustersToDeduplicate.count //TODO: some statistics here on cluster, would be useful. - val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) +// val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) + val tiledTasks = clustersToDeduplicate.flatMap(p => CartesianTaskSplit.parallelizeCluster(p._1, p._2, tileSize)) + tiledTasks.persist //(StorageLevel.MEMORY_AND_DISK) val tileCount = tiledTasks.count; println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") // val comparator = sc.broadcast(buildWorkComparator) //todo: can we do comparator broadcast variable? + //build (clusterId, Seq(docId)) rdd: val partialEqualityClusters = tiledTasks.flatMap( - v => { + task => { val t0 = java.lang.System.currentTimeMillis() - // log.info("Starting tile task %s".format(v.getTaskId)) val comparator = buildWorkComparator - val res = v.processPairs((a: DocumentWrapper, b: DocumentWrapper) => + val res = task.processPairs((a: DocumentWrapper, b: DocumentWrapper) => comparator.isDuplicate(a.getDocumentMetadata, b.getDocumentMetadata, null)) val time = (java.lang.System.currentTimeMillis() - t0) / 1000.0 - log.info(f"Finishing tile task ${v.getTaskId} in $time%.4f sec") - res + //useful for identification of possible problem. + log.info(f"Finishing tile task ${task.taskId} in $time%.4f sec") + res.map((task.clusterId, _)) } ) - // At this moment we have RDD of (String, Seq[String]) (clusterId and list of 'equal' ids inside this cluster - // let's save it: - // partialEqualityClusters.saveAsObjectFile ("hdfs:///user/axnow/intermediate/pec"); - // //later, in other program - watch, you have tuples out of the box :) - // val loadedPec = sc.objectFile[(String, java.util.List[String])]("intermediate/pec") - // val partialEqualityClusters = loadedPec - // - // - // equalityClusters.persist(StorageLevel.MEMORY_AND_DISK) - // log.info("Got totally "+partialEqualityClusters.count+" partial equality clusters."); - // val filteredSizes = partialEqualityClusters.mapValues(x=>1).reduceByKey(_+_).filter(p=>p._2>500) - // log.info("After filtering "+partialEqualityClusters.count+" equality clusters left."); - // - // val eqclsizes= filteredSizes.collect.sortBy (_._2) - // println("Equality cluster sizes:") - // eqclsizes.foreach(println(_)) - // println("Done.\n\n") - + val finalClusters = partialEqualityClusters.mapValues(List(_)). - reduceByKey(_ ++ _). //one long list of lists of ids + reduceByKey(_ ++ _). //one long list of lists of ids for each cluster map(pair => { val t0 = java.lang.System.currentTimeMillis() - val res = TileTask.coalesceResult(pair._2.asJava) + val res = CartesianTaskSplit.coalesceClusters(pair._2.asJava) val tt = System.currentTimeMillis() - t0; val clusterSize = pair._2.size log.info(f"Finished tile coalesce task. (Cluster,time[s], size): ${pair._1}, ${tt / 1000.0}, ${pair._2.size}") diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java index 726a0bf1..1c096122 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DocumentWrapperKryoRegistrator.java @@ -10,15 +10,11 @@ import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ /** - * - * @author Aleksander Nowinski + * Simple class which registers custom serializers for the documents protocol + * buffer generated classes. + * @author Aleksander Nowinski */ public class DocumentWrapperKryoRegistrator implements KryoRegistrator { diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala new file mode 100644 index 00000000..24ec2cc2 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala @@ -0,0 +1,54 @@ +/* + * This file is part of CoAnSys project. + * Copyright (c) 2012-2015 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ +package pl.edu.icm.coansys.document.deduplication +import pl.edu.icm.coansys.commons.java.DocumentWrapperUtils +import pl.edu.icm.coansys.commons.java.StringTools +import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata + +/** + * Generator for the keys used in early stage of the document deduplication. + */ +class MultiLengthTitleKeyGenerator(val keySizes: Seq[Int]) { + def cleanUpString(title: String): String = { + val normalized = StringTools.normalize(title); + //seems that normalize removes stopwords, which is wrong, and quite expensive + //val normalized = StringTools.removeStopWords(StringTools.normalize(title)); + val res = normalized.replaceAll("\\s+", "") + res + } + + def generateKeys(title: String): Seq[String] = { + val ctitle = cleanUpString(title) + val mlen = keySizes.max + val longestKey = ctitle.zipWithIndex.filter(_._2 % 2 == 0).map(_._1).take(mlen).mkString + keySizes.map(keyLength => longestKey.substring(0, Math.min(keyLength, longestKey.size))).distinct + } + + def generateKeys(document: DocumentMetadata): Seq[String] = { + val title: String = DocumentWrapperUtils.getMainTitle(document) + generateKeys(title) + } +} + + +object MultiLengthTitleKeyGenerator { + def generateKeys(document: DocumentMetadata)(keySizes: Seq[Int]): Seq[String] = { + val generator = new MultiLengthTitleKeyGenerator(keySizes) + generator.generateKeys(document) + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java deleted file mode 100644 index 84130424..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/TileTask.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.ListIterator; -import java.util.Objects; -import java.util.Set; -import java.util.function.BiPredicate; -import java.util.stream.Collectors; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; -import scala.Tuple2; - -/** - * - * @author Aleksander Nowinski - */ -public class TileTask implements Serializable { - - String clusterId; - String taskId; - List rows; - List columns; - - public TileTask(String clusterId, List rows, List columns) { - this.clusterId = clusterId; - this.rows = rows; - this.columns = columns; - } - - public String getClusterId() { - return clusterId; - } - - public String getTaskId() { - return taskId; - } - - @Override - public String toString() { - return "TileTask{" + "clusterId=" + clusterId + ", taskId=" + taskId + ", rows[" + rows.size() + "], columns[" + columns.size() + "]}"; - } - - @Override - public int hashCode() { - int hash = 7; - hash = 89 * hash + Objects.hashCode(this.clusterId); - hash = 89 * hash + Objects.hashCode(this.taskId); - return hash; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - final TileTask other = (TileTask) obj; - if (!Objects.equals(this.clusterId, other.clusterId)) { - return false; - } - if (!Objects.equals(this.taskId, other.taskId)) { - return false; - } - return true; - } - - /** - * Generate list of pairs of the documents, where predicate is conformed, ie - * function passed returned true. The predicate is assumed to be - * symmetrical, so it is executed only once on each pair. - * - * @param equalityTest predicate which defines whether or no two elements - * are considered matching (typically equal) - * @return list of pairs of keys of equal documents (documents where - * equalityTest returned true), wrappet into a tuple with cluster id. - */ - public List>> processPairs(BiPredicate equalityTest) { - List> raw = new ArrayList<>(); - for (DocumentWrapper row : rows) { - List equalColums = columns.stream() - .filter( - column -> (row.getDocumentMetadata().getKey().compareTo(column.getDocumentMetadata().getKey()) < 0) - && (equalityTest.test(row, column)) - ).map(column -> column.getDocumentMetadata().getKey()) - .collect(Collectors.toList()); - if (!equalColums.isEmpty()) { - final List rlist = new ArrayList<>(equalColums); - rlist.add(row.getDocumentMetadata().getKey()); - raw.add(rlist); - } - } - raw = coalesceResult(raw); - return raw.stream().map(list -> new Tuple2<>(clusterId, list)).collect(Collectors.toList()); - } - - /** - * Prepare set of tile tasks representing task of comparing the cluster - * cartesian. It is natural to use this as a flatMap operator. - * - * @param clusterId id of the cluster added to the tasks - * @param docs list of documents to be cross-compared - * @param tileSize desired size of the single tile task - * @return list of tasks to be executed in parallel. - */ - public static List parallelize(String clusterId, Collection docs, int tileSize) { - List d = new ArrayList<>(docs); - Collections.sort(d, (d1, d2) -> d1.getDocumentMetadata().getKey().compareTo(d2.getDocumentMetadata().getKey())); - - int ntiles = docs.size() / tileSize + (docs.size() % tileSize == 0 ? 0 : 1); - List portions[] = new List[ntiles]; - for (int i = 0; i < d.size(); i++) { - int idx = i % ntiles; - if (portions[idx] == null) { - portions[idx] = new ArrayList(); - } - portions[idx].add(d.get(i)); - } - List res = new ArrayList<>(); - - for (int i = 0; i < portions.length; i++) { - List rows = portions[i]; - for (int j = i; j < portions.length; j++) { - List columns = portions[j]; - if (rows != null && columns != null) { - final TileTask ntask = new TileTask(clusterId, rows, columns); - ntask.taskId = String.format("%s_%04d:%04d", clusterId, i, j); - res.add(ntask); - } - } - - } - return res; - } - - /** - * Combine clusters which have non-empty intersection, so result will be - * only separate lists. - * - * @param clusters lists to combine - * @return list of the separate clusters, as - */ - public static List> coalesceResult(List> clusters) { - List> all = new ArrayList<>(); - all.addAll(remapToSets(clusters)); - List> res = new ArrayList<>(); - while (!all.isEmpty()) { - Set current = all.remove(0); - boolean anyChange; - do { - anyChange = false; - ListIterator> li = all.listIterator(); - while (li.hasNext()) { - Set next = li.next(); - if (next.stream().anyMatch(f -> current.contains(f))) { - current.addAll(next); - li.remove(); - anyChange = true; - } - } - } while (anyChange); //necessary, as there may be chain of induced joins - res.add(new ArrayList(current)); - } - return res; - } - - /** - * Method which merges clusters of the identifiers ensuring that afterwards - * all clusters where the same element appears are joined into one. It is - * intended to be used as a reduce operator. - * - * @param r1 first list of clusters - * @param r2 second list of clusters. - * @return list of clusters. - */ - public static List> coalesceResult(List> r1, List> r2) { - List> all = new ArrayList<>(); - all.addAll(r1); - all.addAll(r2); - return coalesceResult(all); - } - - protected static List> remapToSets(List> r2) { - return r2.stream().map(x -> new HashSet<>(x)).collect(Collectors.toList()); - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala new file mode 100644 index 00000000..c4e0f191 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplitSuite.scala @@ -0,0 +1,88 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import org.scalatest.FunSuite +import org.scalatest._ +import pl.edu.icm.coansys.models.DocumentProtos +import pl.edu.icm.coansys.models.DocumentProtos._ + +class CartesianTaskSplitSuite extends FunSuite with GivenWhenThen { + + + def createDocument(key:String , title:String):DocumentWrapper = { + DocumentProtos.DocumentWrapper.newBuilder().setDocumentMetadata( + DocumentProtos.DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( + DocumentProtos.BasicMetadata.newBuilder().addTitle( + DocumentProtos.TextWithLanguage.newBuilder().setText(title))) + ).setRowId(key).build(); + } + + def createDocumentList(size:Int):Seq[DocumentWrapper] = { + (1 to size).map(idx => createDocument(f"key_$idx", f"title_$idx")).toSeq + } + + def crossProduct[T](l1:Seq[T]):Seq[(T,T)] = { + crossProduct(l1, l1) + } + + + def crossProduct[T](l1:Seq[T], l2:Seq[T]):Seq[(T,T)] = { + l1.flatMap(x1=>l2.map((x1,_))) + } + + + + test("Parallelize empty set") { + Given("Empty task list") + When("We parallelise") + val res = CartesianTaskSplit.parallelizeCluster("testCluster", Seq.empty[DocumentWrapper], 10) + Then("result is empty") + assert(res.isEmpty) + } + + test("Parallelize set") { + Given("Set of 5 documents") + val docs = createDocumentList(5) + val clusterId = "testCluster" + When("We parallelise with size equal to doc number") + val res = CartesianTaskSplit.parallelizeCluster(clusterId, docs, docs.size) + Then("result is single item") + assertResult(1)(res.size) + When("We parallelize with large tile size") + val r2 = CartesianTaskSplit.parallelizeCluster(clusterId, docs, docs.size+3) + Then("result is single item") + assertResult(1)(r2.size) + When("We parallelize with large 3") + val r3 = CartesianTaskSplit.parallelizeCluster(clusterId, docs, 3) + Then("result have 4 tasks") + assertResult(4)(r3.size) + And("Each task the same Given clusterId") + assert(r3.forall(_.clusterId==clusterId)) + } + + + +// +// test("All items present in result tasks") { +// Given("Set of 5 documents") +// val docs = createDocumentList(5) +// val clusterId = "testCluster" +// When("We parallelise to size 2") +// val res = CartesianTaskSplit.parallelizeCluster(clusterId, docs,2) +// Then("Expect 9 tasks") +// assertResult(9)(res.size) +// And("Each cartesian pair is present") +// val allPairs = crossProduct(docs.map(_.getDocumentMetadata.getKey)).toSet +// +// val taskPairs = res.flatMap(task=> {crossProduct(task.rows, task.columns)}).toSet +// assertResult(allPairs)(taskPairs) +// +// } + + +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java deleted file mode 100644 index 5045a14c..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/CustomOddsCharsKeyGeneratorTest.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import java.util.Arrays; -import static org.testng.Assert.*; -import org.testng.annotations.Test; - -/** - * - * @author Aleksander Nowinski - */ -public class CustomOddsCharsKeyGeneratorTest { - - public CustomOddsCharsKeyGeneratorTest() { - } - - public void setUp() { - } - - public void tearDown() { - } - - /** - * Test of generateKey method, of class CustomOddsCharsKeyGenerator. - */ - @Test - public void testGenerateKey() { - System.out.println("generateKey"); - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - - assertEquals(instance.getKeySize(), CustomOddsCharsKeyGenerator.DEFAULT_KEY_SIZE); - assertEquals(instance.generateKey(""), ""); - assertEquals(instance.generateKey("Short legged cat"), "soteg"); - assertEquals(instance.generateKey("a \t\t12 domino titles"), "1dmnt"); - instance.setKeySize(2); - assertEquals(instance.generateKey("The eleven elves"), "ee"); - assertEquals(instance.generateKey("404"), "44"); - instance.setKeySize(10); - assertEquals(instance.generateKey("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"), "bbbbbbbbbb"); - } - - @Test - public void testSetKeySizes() { - System.out.println("setKeySizes"); - - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - try { - instance.setKeySizes(null); - fail("Permitted null."); - } catch (IllegalArgumentException iae) { - //this is ok. - } - try { - instance.setKeySizes(new int[0]); - fail("Permitted empty array."); - } catch (IllegalArgumentException iae) { - //this is ok. - } - try { - instance.setKeySizes(new int[]{1, 2, 3, 3}); - fail("Permitted two equal params"); - } catch (IllegalArgumentException iae) { - //this is ok. - } - try { - instance.setKeySizes(new int[]{1, 2, 4, 3}); - fail("Permitted unsorted array"); - } catch (IllegalArgumentException iae) { - //this is ok. - } - int[] a = new int[] {1,3,6,7,8,9}; - instance.setKeySizes(a); - assertTrue(Arrays.equals(a, instance.getKeySizes())); - - } - - @Test - public void testGenerateKeys() { - System.out.println("generateKeys"); - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - instance.setKeySizes(new int[]{2, 3, 5}); - assertTrue(Arrays.equals(new String[]{"", "", ""}, instance.generateKeys(""))); - assertTrue(Arrays.equals(new String[]{"so", "sot", "soteg"}, instance.generateKeys("Short legged cat"))); - assertTrue(Arrays.equals(new String[]{"44", "44", "44"}, instance.generateKeys("404"))); - assertTrue(Arrays.equals(new String[]{"bb", "bbb", "bbbbb"}, instance.generateKeys("\t\tbabsbdbsbd bdbdbdb 1b2bcbadsfads"))); - } - - /** - * Test of cleanUpString method, of class CustomOddsCharsKeyGenerator. - */ - @Test - public void testCleanUpString() { - System.out.println("cleanUpString"); - CustomOddsCharsKeyGenerator instance = new CustomOddsCharsKeyGenerator(); - assertEquals(instance.cleanUpString("test"), "test"); - assertEquals(instance.cleanUpString(".-- test --:+"), "test"); - assertEquals(instance.cleanUpString(".-- test -ab-:+"), "testab"); - assertEquals(instance.cleanUpString(".-- test 2-ab-:+"), "test2ab"); - assertEquals(instance.cleanUpString("\t\n test \t\ttest "), "testtest"); - assertEquals(instance.cleanUpString("test of cat"), "testcat"); - assertEquals(instance.cleanUpString("TeSt oF caT\t\t\n"), "testcat"); - assertEquals(instance.cleanUpString("Koń jak koń"), "konjakkon"); - assertEquals(instance.cleanUpString(" Litera β"), "literabeta"); - } - -} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala new file mode 100644 index 00000000..13fc490c --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGeneratorSuite.scala @@ -0,0 +1,67 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import org.scalatest.FunSuite +import org.scalatest._ + +class MultiLengthTitleKeyGeneratorSuite extends FunSuite with GivenWhenThen { + test("cleaning the string") { + Given("an empty instance") + val instance = new MultiLengthTitleKeyGenerator(3 to 7) + When("empty string is given") + val empty = "" + Then("result should be empty") + assertResult("")(instance.cleanUpString(empty)) + + When("String has varied case") + val varcas = "SomeCaseS" + Then("result should be lowercase") + assertResult("somecases")(instance.cleanUpString(varcas)) + + When("String has spaces") + val spc = "Some spaces" + Then("result should be lowercase, no spaces") + assertResult("somespaces")(instance.cleanUpString(spc)) + + When("String has punctuation") + val pct = "String with \"so called\" - phy - punctuation!" + Then("result have no punctuation nor spaces") + assertResult("stringwithsocalledphypunctuation")(instance.cleanUpString(pct)) + + When("String has some stopwords") + val stopwords = "A the long! of short and tall" + Then("result should contain no stopwords") + assertResult("longshorttall")(instance.cleanUpString(stopwords)) + info("That's all folks!") + } + + test("Building the key set") { + Given("An empty instance with sequence keyset from 1 to 6") + val instance = new MultiLengthTitleKeyGenerator(1 to 6) + When("empty string is given") + val empty = "" + Then("result should be list with single, empty string element.") + assert(instance.generateKeys(empty).size==1) + assert(instance.generateKeys(empty)(0).isEmpty) + + When("Normal string is given") + val normal = "abcdefghijklmnopqr" + Then("result array has appropriate lengths") + val normalRes = instance.generateKeys(normal) + assert(normalRes.map(_.size).toList == (1 to 6).toList) + And("result arrray has proper contents.") + assertResult(List("a", "ac", "ace", "aceg", "acegi", "acegik"))(normalRes.toList) + + When("Short string is given") + val short = "abcdef" + Then("result array has appropriate lengths") + val shortRes = instance.generateKeys(short) + assert(shortRes.map(_.size).toList == (1 to 3).toList) + assertResult(List("a", "ac", "ace"))(shortRes.toList) + } +} diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java deleted file mode 100644 index 1b5ad200..00000000 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/TileTaskNGTest.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ -package pl.edu.icm.coansys.document.deduplication; - -import com.google.common.collect.ContiguousSet; -import com.google.common.collect.DiscreteDomain; -import com.google.common.collect.Range; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import static org.testng.Assert.*; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; -import pl.edu.icm.coansys.models.DocumentProtos; -import pl.edu.icm.coansys.models.DocumentProtos.BasicMetadata; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentMetadata; -import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper; -import scala.Tuple2; - -/** - * - * @author Aleksander Nowinski - */ -public class TileTaskNGTest { - - public TileTaskNGTest() { - } - - @BeforeMethod - public void setUpMethod() throws Exception { - } - - @AfterMethod - public void tearDownMethod() throws Exception { - } - - public DocumentWrapper createDocument(String key, String title) { - return DocumentWrapper.newBuilder().setDocumentMetadata( - DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( - BasicMetadata.newBuilder().addTitle( - DocumentProtos.TextWithLanguage.newBuilder().setText(title))) - ).setRowId(key).build(); - } - - protected List createDocumentList(int size) { - return ContiguousSet.create(Range.closed(1, size), DiscreteDomain.integers()).stream() - .map(i -> createDocument(String.format("key_%03d", i), String.format("title_%03d", i))) - .collect(Collectors.toList()); - } - - /** - * Test of processPairs method, of class TileTask. - */ -// @Test -// @Ignore - public void testProcessPairs() { - System.out.println("processPairs"); - int size = 500; - List documents = createDocumentList(size); - System.out.println("Generated documents:"); - System.out.println(documents.stream().limit(10).map(v -> v.toString()).reduce(((s1, s2) -> s1 + "\n" + s2))); - - TileTask instance = new TileTask("test", documents, documents); - List>> res = instance.processPairs((a, b) -> false); - - assertEquals(res.size(), size); - assertTrue(res.stream().anyMatch(p->p._2.size()==1)); - assertEquals(res.stream().map(p->p._2.get(0)).distinct().count(), size); - - - res = instance.processPairs((a, b) -> true); - assertEquals(res.size(), 1); - assertEquals(res.get(0)._2.size(), size); - - - - assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); - - res = instance.processPairs( - (a, b) -> { - String k1 = a.getDocumentMetadata().getKey(); - String k2 = b.getDocumentMetadata().getKey(); - String k1s = k1.substring(5); - String k2s = k2.substring(5); - return k1s.equals(k2s); - }); -// System.out.println("Generated pairs:"); -// System.out.println(res.stream().map(v -> v.toString()).reduce(((s1, s2) -> s1 + ", " + s2))); - assertTrue(res.stream().allMatch(x->x._1().equals(instance.getClusterId()))); - assertEquals(res.size(), 100); - - res.stream().map(x->x._2).forEach( - x->{ - String ke = x.get(0).substring(5); - assertTrue(x.stream().allMatch(key->key.endsWith(ke))); - } - ); - - } - - - private Set toKeySet(List wrappers) { - return wrappers.stream().map(x->x.getDocumentMetadata().getKey()).collect(Collectors.toSet()); - } - - /** - * Test of parallelize method, of class TileTask. - */ - @Test - public void testParallelize() { - System.out.println("parallelize"); - List docs = createDocumentList(9); - String clusterId = "cluster"; - List res = TileTask.parallelize(clusterId, docs, 200); - assertEquals(res.size(), 1, "Created too many tasks."); - - res = TileTask.parallelize(clusterId, docs, 5); - assertEquals(res.size(), 3, "Created invalid number of tasks."); - Set tile0r = toKeySet(res.get(0).rows); - Set tile0c = toKeySet(res.get(0).columns); - assertEquals(tile0r, tile0c); - - Set tile1r = toKeySet(res.get(1).rows); - Set tile1c = toKeySet(res.get(1).columns); - tile1r.stream().forEach( - key->assertFalse(tile1c.contains(key),"In tile 1 key "+key+" from row appears in columns.") - ); - - res = TileTask.parallelize(clusterId, docs, 2); - assertEquals(res.size(), 15, "Created invalid number tasks."); - res = TileTask.parallelize(clusterId, docs, 1); - assertEquals(res.size(), 45, "Created too many tasks."); - } - - /** - * Test of coalesceResult method, of class TileTask. - */ - @Test - public void testCoalesceResult() { - System.out.println("coalesceResult"); - List> r1 = Arrays.asList(new List[] { - Arrays.asList(new String[]{"a", "b"}), - Arrays.asList(new String[]{"c", "d"}), - Arrays.asList(new String[]{"e", "f"}), - }); - List> r2 = Arrays.asList(new List[] { - Arrays.asList(new String[]{"a", "c"}), - }); - - List> result = TileTask.coalesceResult(r1, r2); - assertEquals(2, result.size()); - List l1 = result.get(0); - Collections.sort(l1); - assertEquals(l1, - Arrays.asList("a", "b", "c", "d")); - } - - -} From cd1014cb92a6c804bb119a222aa303346a32c0d2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Sun, 23 Jul 2017 23:01:44 +0200 Subject: [PATCH 19/20] Scala version. --- .../deduplication-document-spark-impl/pom.xml | 15 +- .../deduplication/CartesianTaskSplit.scala | 28 +++- .../deduplication/DeduplicateDocuments.scala | 158 ++++++++++++------ .../MultiLengthTitleKeyGenerator.scala | 2 +- .../DeduplicateDocumentTest.scala | 124 ++++++++++++++ 5 files changed, 264 insertions(+), 63 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala diff --git a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml index aa9a9145..4abc9031 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-impl/pom.xml @@ -10,7 +10,12 @@ deduplication-document-spark-impl jar Deduplication - Document - SparkVersion - Implementation - + + + GNU AFFERO GENERAL PUBLIC LICENSE, Version 3 (AGPL-3.0) + http://opensource.org/licenses/AGPL-3.0 + + ssh-cypisek @@ -22,6 +27,8 @@ src/main/scala src/test/scala + + net.alchim31.maven @@ -181,5 +188,11 @@ 3.0.1 test + + com.holdenkarau + spark-testing-base_2.11 + 1.6.0_0.7.2 + test + diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala index 2bc76c42..5404d932 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/CartesianTaskSplit.scala @@ -1,9 +1,20 @@ /* - * To change this license header, choose License Headers in Project Properties. - * To change this template file, choose Tools | Templates - * and open the template in the editor. - */ + * This file is part of CoAnSys project. + * Copyright (c) 2012-2017 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ package pl.edu.icm.coansys.document.deduplication import pl.edu.icm.coansys.models.DocumentProtos.DocumentWrapper @@ -43,6 +54,7 @@ class CartesianTaskSplit( } object CartesianTaskSplit { + val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) /** * Combine clusters which have non-empty intersection, so result will be * only separate lists. @@ -70,18 +82,18 @@ object CartesianTaskSplit { /** Split one large cluster into parallel tasks of the given size. */ def parallelizeCluster(clusterId: String, documents: Iterable[DocumentWrapper], tileSize: Int): Seq[CartesianTaskSplit] = { - println(f"Document count: ${documents.size}, tile size $tileSize") + log.info(f"Document count: ${documents.size}, tile size $tileSize") val ntiles = documents.size/tileSize + (if(documents.size % tileSize>0) 1 else 0) println(f"ntiles: $ntiles") - val sdoc = documents.toSeq.sorted(Ordering.by[DocumentWrapper, String](_.getDocumentMetadata.getKey)) - val groupedDocs = sdoc.zipWithIndex.map(docidx => (docidx._2%ntiles, docidx._1)).groupBy[Int](_._1).mapValues(_.map(_._2)) + val sdoc = documents.toVector.sorted(Ordering.by[DocumentWrapper, String](_.getDocumentMetadata.getKey)) + val groupedDocs = sdoc.zipWithIndex.map(docidx => (docidx._2%ntiles, docidx._1)).groupBy[Int](_._1).mapValues(_.map(_._2).toVector).toVector val res = groupedDocs.flatMap(kv => groupedDocs.map(kvin => new CartesianTaskSplit( clusterId, f"${clusterId}_${kv._1}:${kv._2}",kv._2, kvin._2 ) ) ) - res.toSeq + res } } diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala index 378c8e1f..0e4c59e7 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocuments.scala @@ -1,3 +1,21 @@ +/* + * This file is part of CoAnSys project. + * Copyright (c) 2012-2017 ICM-UW + * + * CoAnSys is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + + * CoAnSys is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with CoAnSys. If not, see . + */ + package pl.edu.icm.coansys.document.deduplication import scala.collection.JavaConversions._ import org.apache.spark.SparkContext @@ -18,13 +36,15 @@ import pl.edu.icm.coansys.document.deduplication.merge.DuplicatesMerger import pl.edu.icm.coansys.models.DocumentProtos import pl.edu.icm.coansys.models.DocumentProtos._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import pl.edu.icm.coansys.deduplication.document.comparator.VotesProductComparator import pl.edu.icm.coansys.deduplication.document.comparator.WorkComparator import scala.collection.mutable.ListBuffer import pl.edu.icm.coansys.document.deduplication._ import scala.collection.JavaConverters._ +/** Main application for the deduplication of the documents. + * + */ object DeduplicateDocuments { val log = org.slf4j.LoggerFactory.getLogger(getClass().getName()) @@ -47,6 +67,7 @@ object DeduplicateDocuments { } } + def calculateKeys(doc: DocumentMetadata, initialClusteringKeySize: Int, maximumClusteringKeySize: Int): Seq[String] = { val keySizes = initialClusteringKeySize to maximumClusteringKeySize var res = MultiLengthTitleKeyGenerator.generateKeys(doc)(keySizes) @@ -58,7 +79,9 @@ object DeduplicateDocuments { /** * Group items into large clusters, within which detailed analysis will be - * held. Items are grouped by keys generated from the normalised titles. + * held. + * + * Items are grouped by keys generated from the normalised titles. * If the cluster is too big, then longer keys are used, so smaller clusters are * generated. Treshold is maximumClusterSize. * @@ -73,7 +96,7 @@ object DeduplicateDocuments { doc.getDocumentMetadata(), initialClusteringKeySize, maximumClusteringKeySize)) //we loose documents here, ony ids are preseved val clusterDoc = idClusterKeys.flatMap(kv => kv._2.map(idcluster => (idcluster, kv._1))) // (clusterId => docId) val clusterSizes = idClusterKeys.flatMap(x => (x._2.map(y => (y, 1)))).reduceByKey(_ + _) //(clusterId => clusterSize) - + //build rdd (docId, (clusterId, clusterSize) ) val docClustersWithSizes = clusterDoc.join(clusterSizes).map(p => (p._2._1, (p._1, p._2._2))) //build rdd - (docId, clusterId) @@ -101,12 +124,21 @@ object DeduplicateDocuments { res } + /** + * Merge the documents using appropriate document merger. + */ def mergeDocuments(docs: List[DocumentWrapper]): DocumentWrapper = { val merger = buildDocumentsMerger() val merged = merger.merge(docs); merged } + /** + * Defines comparator according to the weights resulting from experiments. + * + * This is reimplementation of the original Spring XML bean definition, which + * was unnecessary complication at this moment. + */ def buildWorkComparator(): WorkComparator = { val result = new VotesProductComparator; result.setMinVotersWeightRequired(1.5f) @@ -165,18 +197,60 @@ object DeduplicateDocuments { result; } + case class Config( inputFile: String = "", outputFile: String = "", dumpClusters: Boolean = false, keySizeMin: Int = 5, - keySizeMax: Int = 10, + keySizeMax: Int = 15, clusterSizeMax: Int = 500, - tileSize: Int = 25 + tileSize: Int = 25, + filterInvalidDocuments: Boolean = false, + removeDuplicateDocuments: Boolean = false ) - /** - * @param args the command line arguments + + /** Load the documents from the given sequence file, do the optional + * cleanups. + * */ + def loadDocuments( sc: SparkContext, file: String, + filterInvalid: Boolean, removeDoubles: Boolean):RDD[(String, DocumentWrapper)] = { + val rawbytes = sc.sequenceFile[String, BytesWritable](file).mapValues(_.copyBytes) + println("Loaded raw bytes.") + + val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) + + //fix invalid documents: + val fixedWrappers = if (filterInvalid) { + val x = dirtyWrappers.filter(w => isValidDocument(w._2)) + val afterSize = x.count; + val preSize = dirtyWrappers.count + log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") + x + } else { + dirtyWrappers + } + + if (removeDoubles) { + fixedWrappers.reduceByKey((x, y) => y) + } else { + fixedWrappers + } + } + + /** Debug method to printout top clusters. */ + def printTopClusters(finalClusters:RDD[(String, Seq[String])], count:Int):Unit = { + val finclSizes = finalClusters.mapValues(_.size).takeOrdered(100)(Ordering[Int].on(-_._2)) + println("Top 100 cluster sizes:") + finclSizes.foreach(println(_)) + println("-----\n\n") + + } + + + + def main(args: Array[String]): Unit = { val fixInvalidDocuments = true; val removeDoubles = true; @@ -185,10 +259,13 @@ object DeduplicateDocuments { val parser = new scopt.OptionParser[Config]("CoAnSys Deduplicate Documents") { head("Deduplicate documents", "0.1") - -// opt[Unit]('c', "dump-clusters").action((x, c) => -// c.copy(dumpClusters = true)).text("dump similarity clusters into debug/clusters hdfs dir during run") - + + opt[Unit]('f', "filter-invalid").action((x, c) => + c.copy(filterInvalidDocuments = true)).text("filter invalid (empty) documents before run.") + + opt[Unit]('d', "remove-doubles").action((x, c) => + c.copy(removeDuplicateDocuments = true)).text("filter out duplicates sharing the same key before processing.") + opt[Int]("cluster-key-min").abbr("kmn").action((x, c) => c.copy(keySizeMin = x)). validate(x => if (x >= 2) success @@ -222,12 +299,13 @@ object DeduplicateDocuments { config case None => // arguments are bad, error message will have been displayed - println("No config.") + println("No config, aborting.") return } println("Creating context...") + //required to operate protobuf correctly val conf = new SparkConf() .setAppName("Document deduplication") .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") @@ -242,52 +320,27 @@ object DeduplicateDocuments { // val inputDocuments = "/user/kura/curr-res-navigator/hbase-sf-out/DOCUMENT" // val inputDocuments = "/user/kura/curr-res-navigator-no-blogs/hbase-sf-out/DOCUMENT" val outputDocuments = cfg.outputFile - val tileSize = cfg.tileSize - val initialClusteringKeySize = cfg.keySizeMin - val maximumClusteringKeySize = cfg.keySizeMax - val maximumClusterSize = cfg.clusterSizeMax - - val rawbytes = sc.sequenceFile[String, BytesWritable](inputDocuments).mapValues(_.copyBytes) - println("Loaded raw bytes.") - - val dirtyWrappers = rawbytes.mapValues(b => DocumentProtos.DocumentWrapper.parseFrom(b)) - - //fix invalid documents: - val fixedWrappers = if (fixInvalidDocuments) { - val x = dirtyWrappers.filter(w => isValidDocument(w._2)) - val afterSize = x.count; - val preSize = dirtyWrappers.count - log.info(f"Filtering invalid documents done, before filtering: $preSize and after filtering $afterSize documents left.") - x - } else { - dirtyWrappers - } - - val wrappers = if (removeDoubles) { - fixedWrappers.reduceByKey((x, y) => y) - } else { - fixedWrappers - } - + + + //load documents + val wrappers = loadDocuments(sc, inputDocuments, cfg.filterInvalidDocuments, cfg.removeDuplicateDocuments) val initialSize = wrappers.count println(f"Starting processing with $initialSize documents.") - val initialGroups = prepareInitialClustering(wrappers, initialClusteringKeySize, maximumClusteringKeySize, maximumClusterSize) - initialGroups.persist //(StorageLevel.MEMORY_AND_DISK) + val initialGroups = prepareInitialClustering(wrappers, cfg.keySizeMin, cfg.keySizeMax, cfg.clusterSizeMax) + initialGroups.persist val clustersToDeduplicate = initialGroups.filter(t => t._2.size > 1) val initialClusterCount = clustersToDeduplicate.count //TODO: some statistics here on cluster, would be useful. -// val tiledTasks = clustersToDeduplicate.flatMap(p => TileTask.parallelize(p._1, p._2, tileSize)) - val tiledTasks = clustersToDeduplicate.flatMap(p => CartesianTaskSplit.parallelizeCluster(p._1, p._2, tileSize)) + val tiledTasks = clustersToDeduplicate.flatMap(p => CartesianTaskSplit.parallelizeCluster(p._1, p._2, cfg.tileSize)) - tiledTasks.persist //(StorageLevel.MEMORY_AND_DISK) + tiledTasks.persist val tileCount = tiledTasks.count; println(f"Prepared $initialClusterCount clusters, and then split it to $tileCount tiles") - // val comparator = sc.broadcast(buildWorkComparator) //todo: can we do comparator broadcast variable? - + //build (clusterId, Seq(docId)) rdd: val partialEqualityClusters = tiledTasks.flatMap( task => { @@ -319,16 +372,15 @@ object DeduplicateDocuments { cl.zipWithIndex.map(q => (cid + f"_${q._2}%03d", q._1)) } ) + //now we got all the items in place - finalClusters.persist(StorageLevel.MEMORY_AND_DISK_SER); - val finclSizes = finalClusters.mapValues(_.size).takeOrdered(100)(Ordering[Int].on(-_._2)) - println("Top 100 cluster sizes:") - finclSizes.foreach(println(_)) - println("-----\n\n") + finalClusters.persist + + printTopClusters(finalClusters, 100) //count clusters, documents in clusters and number of documents to be deduced: val finalClusterCount = finalClusters.count - val documentInFinalClusterCount = finalClusters.map(_._2.size).reduce(_ + _) + val documentInFinalClusterCount = finalClusters.map(_._2.size).fold(0)(_ + _) val documentRemovedDuringClusteringCount = documentInFinalClusterCount - finalClusterCount println(f"Finally created $finalClusterCount clusters, containing $documentInFinalClusterCount documents, $documentRemovedDuringClusteringCount documents will be removed.") @@ -337,7 +389,7 @@ object DeduplicateDocuments { map(v => (v._2, v._1)) val documentWrappersPrepared = wrappers.leftOuterJoin(docIdWithClusterId); val mergedDocuments = documentWrappersPrepared.filter(_._2._2.isDefined). - map(x => (x._2._2, List(x._2._1))).reduceByKey(_ ++ _). //get lists of cluster documents by cluster id + map(x => (x._2._2, List(x._2._1))).foldByKey(List())(_ ++ _). //get lists of cluster documents by cluster id map(kv => { val doc = mergeDocuments(kv._2) (doc.getDocumentMetadata.getKey, doc) diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala index 24ec2cc2..c1ae7b8e 100644 --- a/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/main/scala/pl/edu/icm/coansys/document/deduplication/MultiLengthTitleKeyGenerator.scala @@ -1,6 +1,6 @@ /* * This file is part of CoAnSys project. - * Copyright (c) 2012-2015 ICM-UW + * Copyright (c) 2012-2017 ICM-UW * * CoAnSys is free software: you can redistribute it and/or modify * it under the terms of the GNU Affero General Public License as published by diff --git a/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala new file mode 100644 index 00000000..39b358c0 --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-impl/src/test/scala/pl/edu/icm/coansys/document/deduplication/DeduplicateDocumentTest.scala @@ -0,0 +1,124 @@ +/* + * To change this license header, choose License Headers in Project Properties. + * To change this template file, choose Tools | Templates + * and open the template in the editor. + */ + +package pl.edu.icm.coansys.document.deduplication + +import com.holdenkarau.spark.testing.SharedSparkContext +import org.scalatest.FunSuite +import org.scalatest.GivenWhenThen +import pl.edu.icm.coansys.models.DocumentProtos +import pl.edu.icm.coansys.models.DocumentProtos._ + +class DeduplicateDocumentTest extends FunSuite with GivenWhenThen with SharedSparkContext { + + test("docaument validation") { + Given("Empty document") + val doc = DocumentWrapper.newBuilder().setRowId("test").build; + When("We validate") + Then("Document is invalid") + assert(!DeduplicateDocuments.isValidDocument(doc)) + Given("Doc with empty metadata") + val doc2 = DocumentWrapper.newBuilder(doc).setDocumentMetadata( + DocumentMetadata.newBuilder + .setBasicMetadata(BasicMetadata.newBuilder.build) + .setKey("Key") + .build + ).build + When("We test if it is valid") + + Then("It is not valid") + assert(!DeduplicateDocuments.isValidDocument(doc2)) + + Given("Doc with title ") + val doc3 = DocumentWrapper.newBuilder(doc2).setDocumentMetadata( + DocumentMetadata.newBuilder() + .setBasicMetadata(BasicMetadata.newBuilder().addTitle(TextWithLanguage.newBuilder.setText("Title")).build) + .setKey("key") + .build + ) + .build + When("We test if it is valid: ") + assert(DeduplicateDocuments.isValidDocument(doc3)) + Then("It is not valid") + } + + test("Sample with spark context") { + Given("RDD of sequence 1 to n (n=100)") + val n = 100 + val rdd = sc.parallelize(1 to n) + When("We sum") + val sum = rdd.sum + Then("result is n*(n+1)/2") + assertResult(n * (n + 1) / 2)(sum) + + } + + + def createDocument(key:String , title:String):DocumentWrapper = { + DocumentProtos.DocumentWrapper.newBuilder().setDocumentMetadata( + DocumentProtos.DocumentMetadata.newBuilder().setKey(key).setBasicMetadata( + DocumentProtos.BasicMetadata.newBuilder().addTitle( + DocumentProtos.TextWithLanguage.newBuilder().setText(title))) + ).setRowId(key).build(); + } + + + test("Initial clustering test:") { + Given("Data set has the same title begninnings") + + val d3 = (1 to 10).map(x=> createDocument("id_aaa"+x, "aaa")) + val d4 = (1 to 10).map(x=> createDocument("id_aaaa"+x, "aaaa")) + val d5 = (1 to 10).map(x=> createDocument("id_aaaaa"+x, "aaaaa")) + val d12 = (1 to 10).map(x=> createDocument("id_aaaaaaaaaaa"+x, "aaaaaaaaaa"+x)) + val docs = List()++d3++d4++d5++d12; + val input = sc.parallelize(docs).map(doc=> (doc.getRowId, doc)) + When("We build clustering with short key") + val r1 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 2, 20) + Then("We get only one cluster, with all documents:") + val r1c = r1.collect + assert(r1c.size==1) + assert(r1c(0)._2.size==40) + And("Key is 1st and 3rd letter") + assert(r1c(0)._1=="aa") + When("We build clustering with variable key 2-3") + val r2 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 3, 10) + Then("We get only two clusters:") + val r2c = r2.collect + assert(r2c.size==2) + val r2cm = r2c.toMap + assert(r2cm("aa").size==20) + assert(r2cm("aaa").size==20) +// r2c.flatMap(_._2).map(_.getKey()) +// + When("We build clustering with variable key 2-5") + val r3 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 5, 10) + Then("We get 3 clusters:") + val r3c = r3.collect + assert(r3c.size==3) + val r3cm = r3c.toMap + assert(r3cm("aa").size==20) + assert(r3cm("aaa").size==10) + assert(r3cm("aaaa").size==10) + + When("We build clustering with variable key 2-6") + val r4 = DeduplicateDocuments.prepareInitialClustering(input, 2, + 6, 9) + Then("We get 11 clusters:") + val r4c = r4.collect + assert(r4c.size==11) + val r4cm = r4c.toMap + assert(r4cm("aa").size==20) + assert(r4cm("aaa").size==10) + assert(r4cm("aaaaa2").size==1) + assert(r4cm("aaaaa1").size==2) + + + } + +} From 221cd52f3463a2004d1ea5a60234ae7f59e560b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aleksander=20Nowi=C5=84ski?= Date: Mon, 24 Jul 2017 11:49:28 +0200 Subject: [PATCH 20/20] Fixed oozie workflow building. Cleaning up project files. Fixed workflow building for oozie. --- .gitignore | 2 + .../pom.xml | 10 +- .../src/main/oozie/workflow.xml | 101 ++++++++++++++++++ .../deduplication-document-impl/pom.xml | 17 +++ pom.xml | 1 + 5 files changed, 129 insertions(+), 2 deletions(-) create mode 100644 deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml diff --git a/.gitignore b/.gitignore index ad354819..7dc60f54 100644 --- a/.gitignore +++ b/.gitignore @@ -10,6 +10,7 @@ # Netbeans files # nb-configuration.xml +nbaction.xml # IntelliJ IDEA files # .idea @@ -43,3 +44,4 @@ dependency-reduced-pom.xml /affiliation-organization-matching/affiliation-organization-matching-workflow/src/main/oozie/workflow.xml /deduplication-organization/deduplication-organization-workflow/src/main/oozie/workflow.xml +/deduplication-document-spark/deduplication-document-spark-impl/nbproject/ diff --git a/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml index e1a28320..1c5222a6 100644 --- a/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml +++ b/deduplication-document-spark/deduplication-document-spark-workflow/pom.xml @@ -1,6 +1,6 @@ - deduplication-document + deduplication-document-spark pl.edu.icm.coansys 1.11-SNAPSHOT @@ -11,6 +11,12 @@ UTF-8 + + + GNU AFFERO GENERAL PUBLIC LICENSE, Version 3 (AGPL-3.0) + http://opensource.org/licenses/AGPL-3.0 + + @@ -23,7 +29,7 @@ pl.edu.icm.coansys - deduplication-document-impl + deduplication-document-spark-impl ${project.version} diff --git a/deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml b/deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml new file mode 100644 index 00000000..02d2053a --- /dev/null +++ b/deduplication-document-spark/deduplication-document-spark-workflow/src/main/oozie/workflow.xml @@ -0,0 +1,101 @@ + + + + + jobTracker + + + nameNode + + + queueName + default + + + input + ${inputSeqFile} + + + output + ${outputSeqFile} + + + + sparkExecutorMemory + 128G + memory for individual executor + + + sparkExecutorCores + 16 + number of cores used by single executor + + + sparkExecutorsNumber + 16 + total number of executors + + + sparkHistoryServer + http://spark-m2.vls.icm.edu.pl:18080 + Address of spark history server + + + sparkEventLogDir + hdfs:/user/spark/applicationHistory + Directory for spark events logging + + + + + + + + + + ${jobTracker} + ${nameNode} + + + + + + yarn-cluster + cluster + document-deduplication-spark + + pl.edu.icm.coansys.document.deduplication.DeduplicateDocuments + + ${comacWfPath}/lib/citation-matching-coansys-code-${project.version}.jar + + --conf spark.network.timeout=10000000 --conf spark.executor.heartbeatInterval=10000000 --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} --num-executors ${sparkExecutorsNumber} --conf spark.yarn.historyServer.address=${sparkHistoryServer} --conf spark.eventLog.dir=${sparkEventLogDir} --conf spark.eventLog.enabled=true + f + d + -ts + 50 + ${input} + ${output} + + + + + + + + Workflow failed, error message [${wf:errorMessage(wf:lastErrorNode())}] + + + diff --git a/deduplication-document/deduplication-document-impl/pom.xml b/deduplication-document/deduplication-document-impl/pom.xml index b3b13ce5..4fb1514c 100644 --- a/deduplication-document/deduplication-document-impl/pom.xml +++ b/deduplication-document/deduplication-document-impl/pom.xml @@ -35,6 +35,23 @@ 1.8 + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + + + + package + + shade + + + + + diff --git a/pom.xml b/pom.xml index bff9ad32..4734b1a6 100644 --- a/pom.xml +++ b/pom.xml @@ -198,6 +198,7 @@ citation-matching commons deduplication-document + deduplication-document-spark deduplication-organization disambiguation disambiguation-author