Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Resolves #195: Codify creation of standard derivatives into apps #222

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -595,6 +595,12 @@
<artifactId>lintools-datatypes</artifactId>
<version>1.0.0</version>
</dependency>
<!-- https://mvnrepository.com/artifact/org.rogach/scallop -->
<dependency>
<groupId>org.rogach</groupId>
<artifactId>scallop_2.12</artifactId>
<version>3.1.2</version>
</dependency>
<!--START issue #113-->
<dependency>
<groupId>joda-time</groupId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
package io.archivesunleashed.app
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add standard license header to this and all files.


import java.nio.file.Path

import io.archivesunleashed._
import io.archivesunleashed.matchbox.ExtractDomain
import org.apache.log4j.Logger
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.rdd.RDD
import org.rogach.scallop._

object DomainFrequencyExtractor {
val log = Logger.getLogger(getClass().getName())

class Conf(args: Seq[String]) extends ScallopConf(args) {
mainOptions = Seq(input, output)
val input = opt[String](descr = "input path", required = true)
val output = opt[String](descr = "output path", required = true)
verify()
}

def apply(records: RDD[ArchiveRecord]) = {
records
.keepValidPages()
.map(r => ExtractDomain(r.getUrl))
.countItems()
}

def main(argv: Array[String]): Unit = {
var args = new Conf(argv)

log.info("Domain frequency extractor")
log.info("Input: " + args.input())
log.info("Output: " + args.output())

val conf = new SparkConf().setAppName("Domain frequency extractor")
conf.set("spark.driver.allowMultipleContexts", "true")
val sc = new SparkContext(conf)

RecordLoader.loadArchives(args.input(), sc).saveAsTextFile(args.output())
}
}
33 changes: 33 additions & 0 deletions src/main/scala/io/archivesunleashed/app/DomainGraphExtractor.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
package io.archivesunleashed.app

import io.archivesunleashed._
import io.archivesunleashed.app.DomainFrequencyExtractor.{Conf, log}
import io.archivesunleashed.matchbox.{ExtractDomain, ExtractLinks}
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.rdd.RDD

object DomainGraphExtractor {
def apply(records: RDD[ArchiveRecord]) = {
records
.keepValidPages()
.map(r => (r.getCrawlDate, ExtractLinks(r.getUrl, r.getContentString)))
.flatMap(r => r._2.map(f => (r._1, ExtractDomain(f._1).replaceAll("^\\\\s*www\\\\.", ""), ExtractDomain(f._2).replaceAll("^\\\\s*www\\\\.", ""))))
.filter(r => r._2 != "" && r._3 != "")
.countItems()
.filter(r => r._2 > 5)
}

def main(argv: Array[String]): Unit = {
var args = new Conf(argv)

log.info("Domain graph extractor")
log.info("Input: " + args.input())
log.info("Output: " + args.output())

val conf = new SparkConf().setAppName("Domain graph extractor")
conf.set("spark.driver.allowMultipleContexts", "true")
val sc = new SparkContext(conf)

RecordLoader.loadArchives(args.input(), sc).saveAsTextFile(args.output())
}
}
29 changes: 29 additions & 0 deletions src/main/scala/io/archivesunleashed/app/PlainTextExtractor.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
package io.archivesunleashed.app

import io.archivesunleashed.app.DomainFrequencyExtractor.{Conf, log}
import io.archivesunleashed.{ArchiveRecord, RecordLoader}
import io.archivesunleashed.matchbox.RemoveHTML
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.rdd.RDD

object PlainTextExtractor {
def apply(records: RDD[ArchiveRecord]) = {
records
.keepValidPages()
.map(r => (r.getCrawlDate, r.getDomain, r.getUrl, RemoveHTML(r.getContentString)))
}

def main(argv: Array[String]): Unit = {
var args = new Conf(argv)

log.info("Plain text extractor")
log.info("Input: " + args.input())
log.info("Output: " + args.output())

val conf = new SparkConf().setAppName("Plain text extractor")
conf.set("spark.driver.allowMultipleContexts", "true")
val sc = new SparkContext(conf)

RecordLoader.loadArchives(args.input(), sc).saveAsTextFile(args.output())
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Archives Unleashed Toolkit (AUT):
* An open-source platform for analyzing web archives.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.archivesunleashed.app

import com.google.common.io.Resources
import io.archivesunleashed.RecordLoader
import org.apache.spark.{SparkConf, SparkContext}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import org.scalatest.{BeforeAndAfter, FunSuite}

@RunWith(classOf[JUnitRunner])
class DomainFrequencyExtractorTest extends FunSuite with BeforeAndAfter {
private val arcPath = Resources.getResource("warc/example.warc.gz").getPath
private var sc: SparkContext = _
private val master = "local[4]"
private val appName = "example-spark"

before {
val conf = new SparkConf()
.setMaster(master)
.setAppName(appName)
conf.set("spark.driver.allowMultipleContexts", "true")
sc = new SparkContext(conf)
}

test("extract list of domains with their frequencies in RDD with UDF") {
val examplerdd = RecordLoader.loadArchives(arcPath, sc)
var domainFreq = DomainFrequencyExtractor.apply(examplerdd).collect()

assert(domainFreq(0)._1 == "www.archive.org")
assert(domainFreq(0)._2 == 132)

assert(domainFreq(1)._1 == "deadlists.com")
assert(domainFreq(1)._2 == 2)

assert(domainFreq(2)._1 == "www.hideout.com.br")
assert(domainFreq(2)._2 == 1)
}

after {
if (sc != null) {
sc.stop()
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* Archives Unleashed Toolkit (AUT):
* An open-source platform for analyzing web archives.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.archivesunleashed.app

import com.google.common.io.Resources
import io.archivesunleashed.RecordLoader
import org.apache.spark.{SparkConf, SparkContext}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import org.scalatest.{BeforeAndAfter, FunSuite}

@RunWith(classOf[JUnitRunner])
class DomainGraphExtractorTest extends FunSuite with BeforeAndAfter {
private val arcPath = Resources.getResource("warc/example.warc.gz").getPath
private var sc: SparkContext = _
private val master = "local[4]"
private val appName = "example-spark"

before {
val conf = new SparkConf()
.setMaster(master)
.setAppName(appName)
conf.set("spark.driver.allowMultipleContexts", "true")
sc = new SparkContext(conf)
}

test("extract domain graph in RDD with UDF") {
val examplerdd = RecordLoader.loadArchives(arcPath, sc)
var domainGraph = DomainGraphExtractor.apply(examplerdd).collect()

assert(domainGraph.length == 9)

assert(domainGraph(0)._1._1 == "20080430")
assert(domainGraph(0)._1._2 == "www.archive.org")
assert(domainGraph(0)._1._3 == "www.archive.org")
assert(domainGraph(0)._2 == 305)
}

after {
if (sc != null) {
sc.stop()
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* Archives Unleashed Toolkit (AUT):
* An open-source platform for analyzing web archives.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writi ng, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.archivesunleashed.app

import com.google.common.io.Resources
import io.archivesunleashed.RecordLoader
import org.apache.spark.{SparkConf, SparkContext}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import org.scalatest.{BeforeAndAfter, FunSuite}

@RunWith(classOf[JUnitRunner])
class PlainTextExtractorTest extends FunSuite with BeforeAndAfter {
private val arcPath = Resources.getResource("warc/example.warc.gz").getPath
private var sc: SparkContext = _
private val master = "local[4]"
private val appName = "example-spark"

before {
val conf = new SparkConf()
.setMaster(master)
.setAppName(appName)
conf.set("spark.driver.allowMultipleContexts", "true")
sc = new SparkContext(conf)
}

test("extract plain text from html in RDD with UDF") {
val examplerdd = RecordLoader.loadArchives(arcPath, sc)
var plainText = PlainTextExtractor.apply(examplerdd).collect()

assert(plainText.length == 135)
assert(plainText(0)._1 == "20080430")
assert(plainText(0)._2 == "www.archive.org")
assert(plainText(0)._3 == "http://www.archive.org/")
assert(plainText(0)._4 == "HTTP/1.1 200 OK Date: Wed, 30 Apr 2008 20:48:25 GMT Server: Apache/2.0.54 (Ubuntu) PHP/5.0.5-2ubuntu1.4 mod_ssl/2.0.54 OpenSSL/0.9.7g Last-Modified: Wed, 09 Jan 2008 23:18:29 GMT ETag: \"47ac-16e-4f9e5b40\" Accept-Ranges: bytes Content-Length: 366 Connection: close Content-Type: text/html; charset=UTF-8 Please visit our website at: http://www.archive.org")

}

after {
if (sc != null) {
sc.stop()
}
}
}