├── .gitignore ├── LICENSE ├── README.md ├── bs4_parser.py ├── cc_index_export.py ├── cc_index_word_count.py ├── get-data.sh ├── hostlinks_extract_fastwarc.py ├── hostlinks_to_graph.py ├── html_tag_count.py ├── iana_tld.py ├── json_importer.py ├── linkmap2parquet.py ├── requirements.txt ├── resiliparse_parser.py ├── server_count.py ├── server_count_fastwarc.py ├── server_ip_address.py ├── sparkcc.py ├── sparkcc_fastwarc.py ├── wat_extract_links.py └── word_count.py /.gitignore: -------------------------------------------------------------------------------- 1 | *.py[cod] 2 | 3 | __pycache__ 4 | 5 | # Eclipse project files 6 | .project 7 | .pydevproject 8 | .settings/ 9 | 10 | # Intellij IDEA PyCharm 11 | .idea/ 12 | .idea_modules/ 13 | 14 | # default output folder when running/testing locally 15 | spark-warehouse/ 16 | 17 | # get-data.sh puts data into 18 | crawl-data/ 19 | input/ -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2017 Common Crawl 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ![Common Crawl Logo](https://commoncrawl.org/wp-content/uploads/2016/12/logocommoncrawl.png) 2 | 3 | # Common Crawl PySpark Examples 4 | 5 | This project provides examples how to process the Common Crawl dataset with [Apache Spark](https://spark.apache.org/) and Python: 6 | 7 | + [count HTML tags](./html_tag_count.py) in Common Crawl's raw response data (WARC files) 8 | 9 | + [count web server names](./server_count.py) in Common Crawl's metadata (WAT files or WARC files) 10 | 11 | + list host names and corresponding [IP addresses](./server_ip_address.py) (WAT files or WARC files) 12 | 13 | + [word count](./word_count.py) (term and document frequency) in Common Crawl's extracted text (WET files) 14 | 15 | + [extract links](./wat_extract_links.py) from WAT files and [construct the (host-level) web graph](./hostlinks_to_graph.py) – for further details about the web graphs see the project [cc-webgraph](https://github.com/commoncrawl/cc-webgraph) 16 | 17 | + work with the [columnar URL index](https://commoncrawl.org/2018/03/index-to-warc-files-and-urls-in-columnar-format/) (see also [cc-index-table](https://github.com/commoncrawl/cc-index-table) and the notes about [querying the columnar index](#querying-the-columnar-index)): 18 | 19 | - run a SQL query and [export the result as a table](./cc_index_export.py) 20 | 21 | - select WARC records by a SQL query, parse the HTML, extract the text and [count words](./cc_index_word_count.py). Alternatively, the first step (query the columnar index) can be executed using Amazon Athena. The list of WARC record coordinates (CSV or a table created by a CTAS statement) is then passed via `--csv` or `--input_table_format`) to the Spark job. 22 | 23 | Further information about the examples and available options is shown via the [command-line option](#command-line-options) `--help`. 24 | 25 | ## Implementing a Custom Extractor 26 | 27 | Extending the [CCSparkJob](./sparkcc.py) isn't difficult and for many use cases it is sufficient to override a single method (`process_record`). Have a look at one of the examples, e.g. to [count HTML tags](./html_tag_count.py). 28 | 29 | ## Setup 30 | 31 | To develop and test locally, you will need to install 32 | * Spark, see the [detailed instructions](https://spark.apache.org/docs/latest/), and 33 | * all required Python modules by running 34 | ``` 35 | pip install -r requirements.txt 36 | ``` 37 | * (optionally, and only if you want to query the columnar index) [install S3 support libraries](#installation-of-s3-support-libraries) so that Spark can load the columnar index from S3 38 | 39 | 40 | ## Compatibility and Requirements 41 | 42 | Tested with with Spark 3.2.3, 3.3.2, 3.4.1, 3.5.5 in combination with Python 3.8, 3.9, 3.10, 3.12 and 3.13. See the branch [python-2.7](/commoncrawl/cc-pyspark/tree/python-2.7) if you want to run the job on Python 2.7 and older Spark versions. 43 | 44 | 45 | ## Get Sample Data 46 | 47 | To develop locally, you'll need at least three data files – one for each format used in at least one of the examples. They can be fetched from the following links: 48 | * [warc.gz](https://data.commoncrawl.org/crawl-data/CC-MAIN-2017-13/segments/1490218186353.38/warc/CC-MAIN-20170322212946-00000-ip-10-233-31-227.ec2.internal.warc.gz) 49 | * [wat.gz](https://data.commoncrawl.org/crawl-data/CC-MAIN-2017-13/segments/1490218186353.38/wat/CC-MAIN-20170322212946-00000-ip-10-233-31-227.ec2.internal.warc.wat.gz) 50 | * [wet.gz](https://data.commoncrawl.org/crawl-data/CC-MAIN-2017-13/segments/1490218186353.38/wet/CC-MAIN-20170322212946-00000-ip-10-233-31-227.ec2.internal.warc.wet.gz) 51 | 52 | Alternatively, running `get-data.sh` downloads the sample data. It also writes input files containing 53 | * sample input as `file://` URLs 54 | * all input of one monthly crawl as paths relative to the data bucket base URL `s3://commoncrawl/` resp. `https://data.commoncrawl.org/` – see [authenticated S3 access or access via HTTP](#authenticated-s3-access-or-access-via-http) for more information. 55 | 56 | Note that the sample data is from an older crawl (`CC-MAIN-2017-13` run in March 2017). If you want to use more recent data, please visit the [Common Crawl site](https://commoncrawl.org/the-data/get-started/). 57 | 58 | 59 | ## Process Common Crawl Data on Spark 60 | 61 | CC-PySpark reads the list of input files from a manifest file. Typically, these are Common Crawl WARC, WAT or WET files, but it could be any other type of file, as long it is supported by the class implementing [CCSparkJob](./sparkcc.py). The files can be given as absolute URLs or as paths relative to a base URL (option `--input_base_url`). The URL cat point to a local file (`file://`), to a remote location (typically below `s3://commoncrawl/` resp. `https://data.commoncrawl.org/`). For development and testing, you'd start with local files. 62 | 63 | ### Running locally 64 | 65 | First, point the environment variable `SPARK_HOME` to your Spark installation. 66 | Then submit a job via 67 | 68 | ``` 69 | $SPARK_HOME/bin/spark-submit ./server_count.py \ 70 | --num_output_partitions 1 --log_level WARN \ 71 | ./input/test_warc.txt servernames 72 | ``` 73 | 74 | This will count web server names sent in HTTP response headers for the sample WARC input and store the resulting counts in the SparkSQL table "servernames" in your warehouse location defined by `spark.sql.warehouse.dir` (usually in your working directory as `./spark-warehouse/servernames`). 75 | 76 | The output table can be accessed via SparkSQL, e.g., 77 | 78 | ``` 79 | $SPARK_HOME/bin/pyspark 80 | >>> df = sqlContext.read.parquet("spark-warehouse/servernames") 81 | >>> for row in df.sort(df.val.desc()).take(10): print(row) 82 | ... 83 | Row(key=u'Apache', val=9396) 84 | Row(key=u'nginx', val=4339) 85 | Row(key=u'Microsoft-IIS/7.5', val=3635) 86 | Row(key=u'(no server in HTTP header)', val=3188) 87 | Row(key=u'cloudflare-nginx', val=2743) 88 | Row(key=u'Microsoft-IIS/8.5', val=1459) 89 | Row(key=u'Microsoft-IIS/6.0', val=1324) 90 | Row(key=u'GSE', val=886) 91 | Row(key=u'Apache/2.2.15 (CentOS)', val=827) 92 | Row(key=u'Apache-Coyote/1.1', val=790) 93 | ``` 94 | 95 | But it's also possible to configure a different output format, for example CSV or JSON, see the command-line options. 96 | 97 | See also 98 | * [running the Spark shell and submitting Spark jobs](https://spark.apache.org/docs/latest/#running-the-examples-and-shell) 99 | * [PySpark SQL API](https://spark.apache.org/docs/latest/api/python/pyspark.sql.html) 100 | 101 | 102 | ### Running in Spark cluster over large amounts of data 103 | 104 | As the Common Crawl dataset lives in the Amazon Public Datasets program, you can access and process it on Amazon AWS (in the us-east-1 [AWS region](https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/using-regions-availability-zones.html#concepts-regions)) without incurring any transfer costs. The only cost that you incur is the cost of the machines running your Spark cluster. 105 | 106 | 1. spinning up the Spark cluster: [AWS EMR](https://aws.amazon.com/documentation/emr/) contains a ready-to-use Spark installation but you'll find multiple descriptions on the web how to deploy Spark on a cheap cluster of AWS spot instances. See also [launching Spark on a cluster](https://spark.apache.org/docs/latest/#launching-on-a-cluster). 107 | 108 | 2. choose appropriate cluster-specific settings when [submitting jobs](https://spark.apache.org/docs/latest/submitting-applications.html) and also check for relevant [command-line options](#command-line-options) (e.g., `--num_input_partitions` or `--num_output_partitions`, see below) 109 | 110 | 3. don't forget to deploy all dependencies in the cluster, see [advanced dependency management](https://spark.apache.org/docs/latest/submitting-applications.html#advanced-dependency-management) 111 | 112 | 4. also the the file `sparkcc.py` needs to be deployed or added as argument `--py-files sparkcc.py` to `spark-submit`. Note: some of the examples require further Python files as dependencies. 113 | 114 | 115 | ### Command-line options 116 | 117 | All examples show the available command-line options if called with the parameter `--help` or `-h`, e.g. 118 | ``` 119 | $SPARK_HOME/bin/spark-submit ./server_count.py --help 120 | ``` 121 | 122 | #### Overwriting Spark configuration properties 123 | 124 | There are many [Spark configuration properties](https://spark.apache.org/docs/latest/configuration.html) which allow to tune the job execution or output, see for example see [tuning Spark](https://spark.apache.org/docs/latest/tuning.html) or [EMR Spark memory tuning](https://aws.amazon.com/blogs/big-data/best-practices-for-successfully-managing-memory-for-apache-spark-applications-on-amazon-emr/). 125 | 126 | It's possible to overwrite Spark properties when [submitting the job](https://spark.apache.org/docs/latest/submitting-applications.html): 127 | 128 | ``` 129 | $SPARK_HOME/bin/spark-submit \ 130 | --conf spark.sql.warehouse.dir=myWareHouseDir \ 131 | ... (other Spark options, flags, config properties) \ 132 | ./server_count.py \ 133 | ... (program-specific options) 134 | ``` 135 | 136 | #### Authenticated S3 Access or Access Via HTTP 137 | 138 | Since April 2022 there are two ways to access of Common Crawl data: 139 | - using HTTP/HTTPS and the base URL `https://data.commoncrawl.org/` or `https://ds5q9oxwqwsfj.cloudfront.net/` 140 | - using the S3 API to read the bucket `s3://commoncrawl/` requires authentication and makes an Amazon Web Services account mandatory. 141 | 142 | The S3 API is strongly recommended as the most performant access scheme, if the data is processed in the AWS cloud and in the AWS `us-east-1` region. In contrary, if reading the data from outside the AWS cloud, HTTP/HTTPS access is the preferred option. 143 | 144 | Dependent on the chosen access scheme, the data bucket's base URL needs to be passed using the command-line option `--input_base_url`: 145 | - `--input_base_url https://data.commoncrawl.org/` when using HTTP/HTTPS 146 | - `--input_base_url s3://commoncrawl/` when using the S3 API. 147 | 148 | This project uses [boto3](https://boto3.amazonaws.com/v1/documentation/api/latest/index.html) to access WARC, WAT or WET files on `s3://commoncrawl/` over the S3 API. The best way is to ensure that a S3 read-only IAM policy is attached to the the IAM role of the EC2 instances where Common Crawl data is processed, see the [IAM user guide](https://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). If this is no option (or if the processing is not running on AWS), there are various other [options to configure credentials in boto3](https://boto3.amazonaws.com/v1/documentation/api/latest/guide/credentials.html#configuring-credentials). 149 | 150 | Please also note that [querying the columnar index requires S3 access](#supported-access-schemes-for-the-columnar-index). 151 | 152 | 153 | ### Querying the columnar index 154 | 155 | The example tools to query the [columnar URL index](https://commoncrawl.org/2018/03/index-to-warc-files-and-urls-in-columnar-format/) may require additional configuration and setup steps. 156 | 157 | #### Supported access schemes for the columnar index 158 | 159 | Querying the columnar index using cc-pyspark requires authenticated S3 access. There is no support for HTTP/HTTPS access. Please see here for more information about [supported data access schemes](#authenticated-s3-access-or-access-via-http). 160 | 161 | #### Installation of S3 Support Libraries 162 | 163 | While WARC/WAT/WET files are read using boto3, accessing the [columnar URL index](https://commoncrawl.org/2018/03/index-to-warc-files-and-urls-in-columnar-format/) (see option `--query` of CCIndexSparkJob) is done directly by the SparkSQL engine and requires that S3 support libraries are available. These libs are usually provided when the Spark job is run on a Hadoop cluster running on AWS (eg. EMR). However, they may not be provided for any Spark distribution and are usually absent when running Spark locally (not in a Hadoop cluster). In these situations, the easiest way is to add the libs as required packages by adding `--packages org.apache.hadoop:hadoop-aws:3.2.1` to the arguments of `spark-submit`. This will make [Spark manage the dependencies](https://spark.apache.org/docs/latest/submitting-applications.html#advanced-dependency-management) - the hadoop-aws package and transitive dependencies are downloaded as Maven dependencies. Note that the required version of hadoop-aws package depends on the Hadoop version bundled with your Spark installation, e.g., Spark 3.2.1 bundled with Hadoop 3.2 ([spark-3.2.1-bin-hadoop3.2.tgz](https://archive.apache.org/dist/spark/spark-3.2.1/spark-3.2.1-bin-hadoop3.2.tgz)). 164 | 165 | Please also note that: 166 | - the schema of the URL referencing the columnar index depends on the actual S3 file system implementation: it's `s3://` on EMR but `s3a://` when using [s3a](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html#Introducing_the_Hadoop_S3A_client.). 167 | - (since April 2022) only authenticated S3 access is possible. This requires that access to S3 is properly set up. For configuration details, see 168 | [Authorizing access to EMRFS data in Amazon S3](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-plan-credentialsprovider.html) 169 | or [Hadoop-AWS: Authenticating with S3](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html#Authenticating_with_S3). 170 | 171 | Below an example call to count words in 10 WARC records host under the `.is` top-level domain using the `--packages` option: 172 | ``` 173 | $SPARK_HOME/bin/spark-submit \ 174 | --packages org.apache.hadoop:hadoop-aws:3.3.2 \ 175 | ./cc_index_word_count.py \ 176 | --input_base_url s3://commoncrawl/ \ 177 | --query "SELECT url, warc_filename, warc_record_offset, warc_record_length, content_charset FROM ccindex WHERE crawl = 'CC-MAIN-2020-24' AND subset = 'warc' AND url_host_tld = 'is' LIMIT 10" \ 178 | s3a://commoncrawl/cc-index/table/cc-main/warc/ \ 179 | myccindexwordcountoutput \ 180 | --num_output_partitions 1 \ 181 | --output_format json 182 | ``` 183 | 184 | #### Columnar index and schema merging 185 | 186 | The schema of the [columnar URL index](https://commoncrawl.org/2018/03/index-to-warc-files-and-urls-in-columnar-format/) has been extended over time by adding new columns. If you want to query one of the new columns (e.g., `content_languages`), the following [Spark configuration option](#overwriting-spark-configuration-properties) needs to be set: 187 | ``` 188 | --conf spark.sql.parquet.mergeSchema=true 189 | ``` 190 | 191 | However, this option impacts the query performance, so use with care! Please also read [cc-index-table](https://github.com/commoncrawl/cc-index-table) about configuration options to improve the performance of Spark SQL queries. 192 | 193 | Alternatively, it's possible configure the table schema explicitly: 194 | - download the [latest table schema as JSON](https://github.com/commoncrawl/cc-index-table/blob/master/src/main/resources/schema/cc-index-schema-flat.json) 195 | - and use it by adding the command-line argument `--table_schema cc-index-schema-flat.json`. 196 | 197 | 198 | ### Using FastWARC to parse WARC files 199 | 200 | > [FastWARC](https://resiliparse.chatnoir.eu/en/latest/man/fastwarc.html) is a high-performance WARC parsing library for Python written in C++/Cython. The API is inspired in large parts by WARCIO, but does not aim at being a drop-in replacement. 201 | 202 | Replacing [FastWARC](https://resiliparse.chatnoir.eu/en/latest/man/fastwarc.html) can speed up job execution by 25% if little custom computations are done and most of the time is spent for parsing WARC files. 203 | 204 | To use FastWARC 205 | - the job class must inherit from [CCFastWarcSparkJob](./sparkcc_fastwarc.py) instead of [CCSparkJob](./sparkcc.py). See [ServerCountFastWarcJob](./server_count_fastwarc.py) for an example. 206 | - when running the job in a Spark cluster, `sparkcc_fastwarc.py` must be passed via `--py-files` in addition to `sparkcc.py` and further job-specific Python files. See also [running in a Spark cluster](#running-in-spark-cluster-over-large-amounts-of-data). 207 | 208 | Some differences between the warcio and FastWARC APIs are hidden from the user in methods implemented in [CCSparkJob](./sparkcc.py) and [CCFastWarcSparkJob](./sparkcc_fastwarc.py) respectively. These methods allow to access WARC or HTTP headers and the payload stream in a unique way, regardless of whether warcio or FastWARC are used. 209 | 210 | However, it's recommended that you carefully verify that your custom job implementation works in combination with FastWARC. There are subtle differences between the warcio and FastWARC APIs, including the underlying classes (WARC/HTTP headers and stream implementations). In addition, FastWARC does not support for legacy ARC files and does not automatically decode HTTP content and transfer encodings (see [Resiliparse HTTP Tools](https://resiliparse.chatnoir.eu/en/latest/man/parse/http.html#read-chunked-http-payloads)). While content and transfer encodings are already decoded in Common Crawl WARC files, this may not be the case for WARC files from other sources. See also [WARC 1.1 specification, http/https response records](https://iipc.github.io/warc-specifications/specifications/warc-format/warc-1.1/#http-and-https-schemes). 211 | 212 | 213 | ## Credits 214 | 215 | Examples are originally ported from Stephen Merity's [cc-mrjob](https://github.com/commoncrawl/cc-mrjob/) with the following changes and upgrades: 216 | * based on Apache Spark (instead of [mrjob](https://mrjob.readthedocs.io/)) 217 | * [boto3](https://boto3.readthedocs.io/) supporting multi-part download of data from S3 218 | * [warcio](https://github.com/webrecorder/warcio) a Python 2 and Python 3 compatible module for accessing WARC files 219 | 220 | Further inspirations are taken from 221 | * [cosr-back](https://github.com/commonsearch/cosr-back) written by Sylvain Zimmer for [Common Search](https://web.archive.org/web/20171117073653/https://about.commonsearch.org/). You should definitely take a look at it if you need a more sophisticated WARC processor (including an HTML parser for example). 222 | * Mark Litwintschik's blog post [Analysing Petabytes of Websites](https://tech.marksblogg.com/petabytes-of-website-data-spark-emr.html) 223 | 224 | 225 | ## License 226 | 227 | MIT License, as per [LICENSE](./LICENSE) 228 | -------------------------------------------------------------------------------- /bs4_parser.py: -------------------------------------------------------------------------------- 1 | from bs4 import BeautifulSoup 2 | from bs4.dammit import EncodingDetector 3 | 4 | 5 | class HTMLParser(object): 6 | """ 7 | HTML parser using BeautifulSoup4 8 | """ 9 | 10 | def html_to_text(self, html_tree: BeautifulSoup) -> str: 11 | """ 12 | Convert HTML content to plain text using BeautifulSoup4. 13 | 14 | Returns: 15 | str: Extracted plain text with scripts and styles removed 16 | """ 17 | for script in html_tree(['script', 'style']): 18 | script.extract() 19 | text = html_tree.get_text(' ', strip=True) 20 | return text 21 | 22 | def get_html_tree(self, page: bytes, encoding: str=None, features='lxml', **kwargs) -> BeautifulSoup: 23 | """ 24 | Return the HTML tree object 25 | 26 | Args: 27 | page (bytes): Raw HTML content as bytes 28 | encoding (str, optional): Specific character encoding to use. If None, auto-detection is attempted 29 | features: Parser to be used (default='lxml'). Refer https://www.crummy.com/software/BeautifulSoup/bs4/doc/#installing-a-parser for supported parsers. 30 | **kwargs: Additional arguments passed to BeautifulSoup constructor. 31 | Refer here https://www.crummy.com/software/BeautifulSoup/bs4/doc/#bs4.BeautifulSoup for accepted arguments. 32 | 33 | Returns: 34 | BeautifulSoup: HTML tree object 35 | """ 36 | if not encoding: 37 | for encoding in EncodingDetector(page, is_html=True).encodings: 38 | # take the first detected encoding 39 | break 40 | soup = BeautifulSoup(page, features, from_encoding=encoding, **kwargs) 41 | return soup -------------------------------------------------------------------------------- /cc_index_export.py: -------------------------------------------------------------------------------- 1 | from sparkcc import CCIndexSparkJob 2 | 3 | 4 | class CCIndexExportJob(CCIndexSparkJob): 5 | """ Export rows matching a SQL query on the columnar URL index """ 6 | 7 | name = "CCIndexExport" 8 | 9 | 10 | if __name__ == '__main__': 11 | job = CCIndexExportJob() 12 | job.run() 13 | -------------------------------------------------------------------------------- /cc_index_word_count.py: -------------------------------------------------------------------------------- 1 | from collections import Counter 2 | 3 | from sparkcc import CCIndexWarcSparkJob 4 | from word_count import WordCountJob 5 | 6 | 7 | class CCIndexWordCountJob(WordCountJob, CCIndexWarcSparkJob): 8 | """ Word count (frequency list) from WARC records matching a SQL query 9 | on the columnar URL index """ 10 | 11 | name = "CCIndexWordCount" 12 | 13 | records_parsing_failed = None 14 | records_non_html = None 15 | 16 | def add_arguments(self, parser): 17 | super(CCIndexWordCountJob, self).add_arguments(parser) 18 | parser.add_argument( 19 | "--html_parser", default="beautifulsoup", 20 | help="HTML parser: beautifulsoup or resiliparse." 21 | " Make sure to install the correct dependencies for the parser and " 22 | "include the correct parser module (bs4_parser.py for beautifulsoup or resiliparse_parser.py for resiliparse) to the cluster" 23 | ) 24 | 25 | def get_html_parser(self): 26 | try: 27 | if self.args.html_parser == 'beautifulsoup': 28 | from bs4_parser import HTMLParser 29 | return HTMLParser() 30 | elif self.args.html_parser == 'resiliparse': 31 | from resiliparse_parser import HTMLParser 32 | return HTMLParser() 33 | else: 34 | raise ValueError( 35 | "Unknown HTML parser: {}".format(self.args.html_parser) 36 | ) 37 | except ImportError as e: 38 | raise ImportError( 39 | f"Failed to import HTML parser module '{self.args.html_parser}'." 40 | f" Please ensure the module is correctly added to PySpark cluster via `--py-files`: {str(e)}" 41 | ) 42 | 43 | def init_accumulators(self, session): 44 | super(CCIndexWordCountJob, self).init_accumulators(session) 45 | 46 | sc = session.sparkContext 47 | self.records_parse_success = sc.accumulator(0) 48 | self.records_parsing_failed = sc.accumulator(0) 49 | self.records_non_html = sc.accumulator(0) 50 | 51 | def log_accumulators(self, session): 52 | super(CCIndexWordCountJob, self).log_accumulators(session) 53 | 54 | self.log_accumulator(session, self.records_parse_success, 55 | 'records successfully parsed = {}') 56 | self.log_accumulator(session, self.records_parsing_failed, 57 | 'records failed to parse = {}') 58 | self.log_accumulator(session, self.records_non_html, 59 | 'records not HTML = {}') 60 | 61 | @staticmethod 62 | def reduce_by_key_func(a, b): 63 | # sum values of tuple 64 | return ((a[0] + b[0]), (a[1] + b[1])) 65 | 66 | def process_record(self, record): 67 | if not self.is_response_record(record): 68 | # skip over WARC request or metadata records 69 | return 70 | if not self.is_html(record): 71 | self.records_non_html.add(1) 72 | return 73 | 74 | text = "" 75 | try: 76 | page = self.get_payload_stream(record).read() 77 | encoding = self.get_warc_header(record, 'WARC-Identified-Content-Charset') 78 | parser = self.get_html_parser() 79 | html_tree = parser.get_html_tree(page, encoding=encoding) 80 | text = parser.html_to_text(html_tree) 81 | except Exception as e: 82 | self.get_logger().error("Error converting HTML to text for {}: {}", 83 | self.get_warc_header(record, 'WARC-Target-URI'), e) 84 | self.records_parsing_failed.add(1) 85 | self.records_parse_success.add(1) 86 | words = map(lambda w: w.lower(), 87 | WordCountJob.word_pattern.findall(text)) 88 | for word, count in Counter(words).items(): 89 | yield word, (count, 1) 90 | 91 | 92 | if __name__ == '__main__': 93 | job = CCIndexWordCountJob() 94 | job.run() 95 | -------------------------------------------------------------------------------- /get-data.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | CRAWL=CC-MAIN-2017-13 4 | 5 | # base URL used to download the path listings 6 | BASE_URL=https://data.commoncrawl.org 7 | 8 | set -e 9 | 10 | test -d input || mkdir input 11 | 12 | if [ -e input/test.txt ]; then 13 | echo "File input/test.txt already exist" 14 | echo "... delete it to write a new one" 15 | exit 1 16 | fi 17 | 18 | for data_type in warc wat wet; do 19 | 20 | echo "Downloading Common Crawl paths listings (${data_type} files of $CRAWL)..." 21 | 22 | mkdir -p crawl-data/$CRAWL/ 23 | listing=crawl-data/$CRAWL/$data_type.paths.gz 24 | cd crawl-data/$CRAWL/ 25 | wget --timestamping $BASE_URL/$listing 26 | cd - 27 | 28 | echo "Downloading sample ${data_type} file..." 29 | 30 | file=$(gzip -dc $listing | head -1) 31 | mkdir -p $(dirname $file) 32 | cd $(dirname $file) 33 | wget --timestamping $BASE_URL/$file 34 | cd - 35 | 36 | echo "Writing input file listings..." 37 | 38 | input=input/test_${data_type}.txt 39 | echo "Test file: $input" 40 | echo file:$PWD/$file >>$input 41 | 42 | input=input/all_${data_type}_${CRAWL}.txt 43 | echo "All ${data_type} files of ${CRAWL}: $input" 44 | gzip -dc $listing >$input 45 | 46 | done 47 | 48 | -------------------------------------------------------------------------------- /hostlinks_extract_fastwarc.py: -------------------------------------------------------------------------------- 1 | from fastwarc.warc import WarcRecordType 2 | 3 | from wat_extract_links import ExtractHostLinksJob, ExtractLinksJob 4 | from sparkcc_fastwarc import CCFastWarcSparkJob 5 | 6 | class ExtractHostLinksFastWarcJob(CCFastWarcSparkJob, ExtractHostLinksJob): 7 | """Extract links from WAT files, redirects from WARC files, 8 | and sitemap links from robots.txt response records relying on the 9 | FastWARC parser. 10 | Extract the host names, reverse the names (example.com -> com.example) 11 | and save the pairs .""" 12 | 13 | # process only WARC response and metadata (including WAT) records 14 | fastwarc_record_filter = WarcRecordType.metadata | WarcRecordType.response 15 | 16 | def iterate_records(self, warc_uri, archive_iterator): 17 | """Iterate over all WARC records and process them""" 18 | self.processing_robotstxt_warc \ 19 | = ExtractLinksJob.robotstxt_warc_path_pattern.match(warc_uri) 20 | for record in archive_iterator: 21 | for res in self.process_record(record): 22 | yield res 23 | self.records_processed.add(1) 24 | 25 | def process_robotstxt(self, record, stream, _http_status_line): 26 | """Process robots.txt and yield sitemap links""" 27 | line = stream.readline() 28 | while line: 29 | if line == b'\r\n': 30 | # end of HTTP header 31 | break 32 | line = stream.readline() 33 | line = stream.readline(crlf=False) 34 | while line: 35 | m = ExtractLinksJob.robotstxt_sitemap_pattern.match(line) 36 | if m: 37 | sitemap = m.group(1).strip() 38 | try: 39 | sitemap = sitemap.decode('utf-8') 40 | from_robotstxt = record.headers['WARC-Target-URI'] 41 | src_host = ExtractHostLinksJob.get_surt_host(from_robotstxt) 42 | thost = ExtractHostLinksJob.get_surt_host(sitemap) 43 | if thost and src_host and src_host != thost: 44 | yield src_host, thost 45 | except UnicodeError as e: 46 | self.get_logger().warning( 47 | 'URL with unknown encoding: {} - {}'.format( 48 | sitemap, e)) 49 | line = stream.readline(crlf=False) 50 | 51 | 52 | if __name__ == "__main__": 53 | job = ExtractHostLinksFastWarcJob() 54 | job.run() 55 | -------------------------------------------------------------------------------- /hostlinks_to_graph.py: -------------------------------------------------------------------------------- 1 | import idna 2 | import logging 3 | import os 4 | 5 | from sparkcc import CCSparkJob 6 | from pyspark.sql import functions as sqlf 7 | from pyspark.sql.types import BooleanType, LongType, StringType, StructField, StructType 8 | 9 | from iana_tld import iana_tld_list 10 | from wat_extract_links import ExtractHostLinksJob 11 | 12 | 13 | class HostLinksToGraph(CCSparkJob): 14 | """Construct host-level webgraph from table with link pairs 15 | (input is a table with reversed host names).""" 16 | 17 | name = "LinksToGraph" 18 | 19 | def add_arguments(self, parser): 20 | parser.add_argument("--save_as_text", type=str, default=None, 21 | help="Save webgraph also as text on path") 22 | parser.add_argument("--normalize_host_names", action='store_true', 23 | help="Normalize host names: replace Unicode IDNs" 24 | " by their ASCII equivalents") 25 | parser.add_argument("--validate_host_names", action='store_true', 26 | help="Validate host names and skip vertices with" 27 | " invalid name during assignment of vertex IDs") 28 | parser.add_argument("--vertex_partitions", type=int, default=1, 29 | help="Number of partitions to enumerate and store" 30 | " vertices. The default (1 partition) is recommended" 31 | " only for smaller graphs. IDs are represented" 32 | " either as int (if 1 partition) or long (multiple" 33 | " partitions).") 34 | parser.add_argument("--vertex_ids", type=str, 35 | help="Path to table providing hostname - vertex ID" 36 | " mappings. If the mapping exists IDs are read" 37 | " from it, otherwise the mapping is created and" 38 | " also saved as table.") 39 | parser.add_argument("--add_input", type=str, action='append', 40 | help="Additional input table to be merged") 41 | 42 | @staticmethod 43 | def reverse_host(host): 44 | parts = host.split('.') 45 | parts.reverse() 46 | return '.'.join(parts) 47 | 48 | @staticmethod 49 | def reverse_host_is_valid(rev_host): 50 | if rev_host is None: 51 | return False 52 | if '.' not in rev_host: 53 | return False 54 | # fast check for valid top-level domain 55 | # (modules tldextract and tld are too expensive) 56 | tld = rev_host.split('.')[0] 57 | if tld not in iana_tld_list: 58 | logging.debug("No valid tld {} in {}".format(tld, rev_host)) 59 | return False 60 | return True 61 | 62 | @staticmethod 63 | def reverse_host_normalize(rev_host): 64 | parts = rev_host.split('.') 65 | modified = False 66 | for (i, part) in enumerate(parts): 67 | if not ExtractHostLinksJob.host_part_pattern.match(part): 68 | try: 69 | idn = idna.encode(part).decode('ascii') 70 | parts[i] = idn 71 | modified = True 72 | except (idna.IDNAError, idna.core.InvalidCodepoint, UnicodeError, IndexError, Exception): 73 | return None 74 | if modified: 75 | return '.'.join(parts) 76 | return rev_host 77 | 78 | def vertices_assign_ids(self, session, edges): 79 | source = edges.select(edges.s.alias('name')) 80 | target = edges.select(edges.t.alias('name')) 81 | 82 | ids = source.union(target) \ 83 | .distinct() 84 | 85 | if self.args.normalize_host_names: 86 | normalize = sqlf.udf(HostLinksToGraph.reverse_host_normalize, 87 | StringType()) 88 | ids = ids.withColumn('name', normalize(ids['name'])) 89 | ids = ids.dropna().distinct() 90 | 91 | if self.args.validate_host_names: 92 | is_valid = sqlf.udf(HostLinksToGraph.reverse_host_is_valid, 93 | BooleanType()) 94 | ids = ids.filter(is_valid(ids['name'])) 95 | 96 | if self.args.vertex_partitions == 1: 97 | ids = ids \ 98 | .coalesce(1) \ 99 | .sort('name') \ 100 | .withColumn('id', sqlf.monotonically_increasing_id()) 101 | else: 102 | id_rdd = ids.select(ids.name).rdd \ 103 | .map(lambda row: tuple(row)[0]) \ 104 | .sortBy(lambda x: x, True, 105 | self.args.vertex_partitions) \ 106 | .zipWithIndex() 107 | id_schema = StructType([ 108 | StructField("name", StringType(), True), 109 | StructField("id", LongType(), True) 110 | ]) 111 | ids = session.createDataFrame(id_rdd, schema=id_schema) 112 | 113 | if self.args.save_as_text is not None: 114 | ids = ids.persist() 115 | ids.select(sqlf.concat_ws('\t', ids.id, ids.name)) \ 116 | .write \ 117 | .text(os.path.join(self.args.save_as_text, "vertices"), 118 | compression="gzip") 119 | ids.write \ 120 | .format(self.args.output_format) \ 121 | .option("compression", self.args.output_compression) \ 122 | .saveAsTable(self.args.output + '_vertices') 123 | 124 | return ids 125 | 126 | def run_job(self, session): 127 | 128 | # read edges s -> t (host names) 129 | edges = session.read.load(self.args.input) 130 | 131 | if self.args.add_input: 132 | # merge multiple input graphs 133 | for add_input in self.args.add_input: 134 | add_edges = session.read.load(add_input) 135 | edges = edges.union(add_edges) 136 | 137 | # remove duplicates and sort 138 | edges = edges \ 139 | .dropDuplicates() \ 140 | .sortWithinPartitions('s', 't') 141 | 142 | if self.args.vertex_ids is not None: 143 | ids = session.read.load(self.args.vertex_ids) 144 | else: 145 | ids = self.vertices_assign_ids(session, edges) 146 | 147 | edges = edges.join(ids, edges.s == ids.name, 'inner') 148 | edges = edges.select(edges.id.alias('s'), 't') 149 | edges = edges.join(ids, edges.t == ids.name, 'inner') 150 | edges = edges.select('s', edges.id.alias('t')) 151 | edges = edges \ 152 | .coalesce(self.args.num_output_partitions) \ 153 | .sortWithinPartitions('s', 't') 154 | 155 | # remove self-loops 156 | # (must be done after assignment of IDs so that isolated 157 | # nodes/vertices are contained in map 158 | edges = edges.filter(edges.s != edges.t) 159 | 160 | if self.args.save_as_text is not None: 161 | edges = edges.persist() 162 | edges.select(sqlf.concat_ws('\t', edges.s, edges.t)) \ 163 | .write \ 164 | .text(os.path.join(self.args.save_as_text, "edges"), 165 | compression="gzip") 166 | # TODO: save as adjacency list 167 | edges.write \ 168 | .format(self.args.output_format) \ 169 | .option("compression", self.args.output_compression) \ 170 | .saveAsTable(self.args.output + '_edges') 171 | 172 | 173 | if __name__ == "__main__": 174 | job = HostLinksToGraph() 175 | job.run() 176 | -------------------------------------------------------------------------------- /html_tag_count.py: -------------------------------------------------------------------------------- 1 | import re 2 | 3 | from collections import Counter 4 | 5 | from sparkcc import CCSparkJob 6 | 7 | 8 | class TagCountJob(CCSparkJob): 9 | """ Count HTML tag names in Common Crawl WARC files""" 10 | 11 | name = "TagCount" 12 | 13 | # match HTML tags (element names) on binary HTML data 14 | html_tag_pattern = re.compile(b'<([a-z0-9]+)') 15 | 16 | def process_record(self, record): 17 | if not self.is_response_record(record): 18 | # skip over WARC request or metadata records 19 | return 20 | if not self.is_html(record): 21 | # skip non-HTML or unknown content types 22 | return 23 | data = self.get_payload_stream(record).read() 24 | counts = Counter(TagCountJob.html_tag_pattern.findall(data)) 25 | for tag, count in counts.items(): 26 | yield tag.decode('ascii').lower(), count 27 | 28 | 29 | if __name__ == '__main__': 30 | job = TagCountJob() 31 | job.run() 32 | -------------------------------------------------------------------------------- /iana_tld.py: -------------------------------------------------------------------------------- 1 | # https://data.iana.org/TLD/tlds-alpha-by-domain.txt 2 | # (merged TLDs from tld files fetched between March 2017 and May 2021) 3 | 4 | iana_tld_list = frozenset({ 5 | 'aaa', 'aarp', 'abarth', 'abb', 'abbott', 'abbvie', 'abc', 6 | 'able', 'abogado', 'abudhabi', 'ac', 'academy', 'accenture', 7 | 'accountant', 'accountants', 'aco', 'active', 'actor', 'ad', 8 | 'adac', 'ads', 'adult', 'ae', 'aeg', 'aero', 'aetna', 'af', 9 | 'afamilycompany', 'afl', 'africa', 'ag', 'agakhan', 'agency', 10 | 'ai', 'aig', 'aigo', 'airbus', 'airforce', 'airtel', 'akdn', 11 | 'al', 'alfaromeo', 'alibaba', 'alipay', 'allfinanz', 12 | 'allstate', 'ally', 'alsace', 'alstom', 'am', 'amazon', 13 | 'americanexpress', 'americanfamily', 'amex', 'amfam', 'amica', 14 | 'amsterdam', 'analytics', 'android', 'anquan', 'anz', 'ao', 15 | 'aol', 'apartments', 'app', 'apple', 'aq', 'aquarelle', 'ar', 16 | 'arab', 'aramco', 'archi', 'army', 'arpa', 'art', 'arte', 'as', 17 | 'asda', 'asia', 'associates', 'at', 'athleta', 'attorney', 18 | 'au', 'auction', 'audi', 'audible', 'audio', 'auspost', 19 | 'author', 'auto', 'autos', 'avianca', 'aw', 'aws', 'ax', 'axa', 20 | 'az', 'azure', 'ba', 'baby', 'baidu', 'banamex', 21 | 'bananarepublic', 'band', 'bank', 'bar', 'barcelona', 22 | 'barclaycard', 'barclays', 'barefoot', 'bargains', 'baseball', 23 | 'basketball', 'bauhaus', 'bayern', 'bb', 'bbc', 'bbt', 'bbva', 24 | 'bcg', 'bcn', 'bd', 'be', 'beats', 'beauty', 'beer', 'bentley', 25 | 'berlin', 'best', 'bestbuy', 'bet', 'bf', 'bg', 'bh', 'bharti', 26 | 'bi', 'bible', 'bid', 'bike', 'bing', 'bingo', 'bio', 'biz', 27 | 'bj', 'black', 'blackfriday', 'blanco', 'blockbuster', 'blog', 28 | 'bloomberg', 'blue', 'bm', 'bms', 'bmw', 'bn', 'bnl', 29 | 'bnpparibas', 'bo', 'boats', 'boehringer', 'bofa', 'bom', 30 | 'bond', 'boo', 'book', 'booking', 'boots', 'bosch', 'bostik', 31 | 'boston', 'bot', 'boutique', 'box', 'br', 'bradesco', 32 | 'bridgestone', 'broadway', 'broker', 'brother', 'brussels', 33 | 'bs', 'bt', 'budapest', 'bugatti', 'build', 'builders', 34 | 'business', 'buy', 'buzz', 'bv', 'bw', 'by', 'bz', 'bzh', 'ca', 35 | 'cab', 'cafe', 'cal', 'call', 'calvinklein', 'cam', 'camera', 36 | 'camp', 'cancerresearch', 'canon', 'capetown', 'capital', 37 | 'capitalone', 'car', 'caravan', 'cards', 'care', 'career', 38 | 'careers', 'cars', 'cartier', 'casa', 'case', 'caseih', 'cash', 39 | 'casino', 'cat', 'catering', 'catholic', 'cba', 'cbn', 'cbre', 40 | 'cbs', 'cc', 'cd', 'ceb', 'center', 'ceo', 'cern', 'cf', 'cfa', 41 | 'cfd', 'cg', 'ch', 'chanel', 'channel', 'charity', 'chase', 42 | 'chat', 'cheap', 'chintai', 'chloe', 'christmas', 'chrome', 43 | 'chrysler', 'church', 'ci', 'cipriani', 'circle', 'cisco', 44 | 'citadel', 'citi', 'citic', 'city', 'cityeats', 'ck', 'cl', 45 | 'claims', 'cleaning', 'click', 'clinic', 'clinique', 46 | 'clothing', 'cloud', 'club', 'clubmed', 'cm', 'cn', 'co', 47 | 'coach', 'codes', 'coffee', 'college', 'cologne', 'com', 48 | 'comcast', 'commbank', 'community', 'company', 'compare', 49 | 'computer', 'comsec', 'condos', 'construction', 'consulting', 50 | 'contact', 'contractors', 'cooking', 'cookingchannel', 'cool', 51 | 'coop', 'corsica', 'country', 'coupon', 'coupons', 'courses', 52 | 'cpa', 'cr', 'credit', 'creditcard', 'creditunion', 'cricket', 53 | 'crown', 'crs', 'cruise', 'cruises', 'csc', 'cu', 'cuisinella', 54 | 'cv', 'cw', 'cx', 'cy', 'cymru', 'cyou', 'cz', 'dabur', 'dad', 55 | 'dance', 'data', 'date', 'dating', 'datsun', 'day', 'dclk', 56 | 'dds', 'de', 'deal', 'dealer', 'deals', 'degree', 'delivery', 57 | 'dell', 'deloitte', 'delta', 'democrat', 'dental', 'dentist', 58 | 'desi', 'design', 'dev', 'dhl', 'diamonds', 'diet', 'digital', 59 | 'direct', 'directory', 'discount', 'discover', 'dish', 'diy', 60 | 'dj', 'dk', 'dm', 'dnp', 'do', 'docs', 'doctor', 'dodge', 61 | 'dog', 'doha', 'domains', 'dot', 'download', 'drive', 'dtv', 62 | 'dubai', 'duck', 'dunlop', 'duns', 'dupont', 'durban', 'dvag', 63 | 'dvr', 'dz', 'earth', 'eat', 'ec', 'eco', 'edeka', 'edu', 64 | 'education', 'ee', 'eg', 'email', 'emerck', 'energy', 65 | 'engineer', 'engineering', 'enterprises', 'epost', 'epson', 66 | 'equipment', 'er', 'ericsson', 'erni', 'es', 'esq', 'estate', 67 | 'esurance', 'et', 'etisalat', 'eu', 'eurovision', 'eus', 68 | 'events', 'everbank', 'exchange', 'expert', 'exposed', 69 | 'express', 'extraspace', 'fage', 'fail', 'fairwinds', 'faith', 70 | 'family', 'fan', 'fans', 'farm', 'farmers', 'fashion', 'fast', 71 | 'fedex', 'feedback', 'ferrari', 'ferrero', 'fi', 'fiat', 72 | 'fidelity', 'fido', 'film', 'final', 'finance', 'financial', 73 | 'fire', 'firestone', 'firmdale', 'fish', 'fishing', 'fit', 74 | 'fitness', 'fj', 'fk', 'flickr', 'flights', 'flir', 'florist', 75 | 'flowers', 'fly', 'fm', 'fo', 'foo', 'food', 'foodnetwork', 76 | 'football', 'ford', 'forex', 'forsale', 'forum', 'foundation', 77 | 'fox', 'fr', 'free', 'fresenius', 'frl', 'frogans', 78 | 'frontdoor', 'frontier', 'ftr', 'fujitsu', 'fujixerox', 'fun', 79 | 'fund', 'furniture', 'futbol', 'fyi', 'ga', 'gal', 'gallery', 80 | 'gallo', 'gallup', 'game', 'games', 'gap', 'garden', 'gay', 81 | 'gb', 'gbiz', 'gd', 'gdn', 'ge', 'gea', 'gent', 'genting', 82 | 'george', 'gf', 'gg', 'ggee', 'gh', 'gi', 'gift', 'gifts', 83 | 'gives', 'giving', 'gl', 'glade', 'glass', 'gle', 'global', 84 | 'globo', 'gm', 'gmail', 'gmbh', 'gmo', 'gmx', 'gn', 'godaddy', 85 | 'gold', 'goldpoint', 'golf', 'goo', 'goodhands', 'goodyear', 86 | 'goog', 'google', 'gop', 'got', 'gov', 'gp', 'gq', 'gr', 87 | 'grainger', 'graphics', 'gratis', 'green', 'gripe', 'grocery', 88 | 'group', 'gs', 'gt', 'gu', 'guardian', 'gucci', 'guge', 89 | 'guide', 'guitars', 'guru', 'gw', 'gy', 'hair', 'hamburg', 90 | 'hangout', 'haus', 'hbo', 'hdfc', 'hdfcbank', 'health', 91 | 'healthcare', 'help', 'helsinki', 'here', 'hermes', 'hgtv', 92 | 'hiphop', 'hisamitsu', 'hitachi', 'hiv', 'hk', 'hkt', 'hm', 93 | 'hn', 'hockey', 'holdings', 'holiday', 'homedepot', 94 | 'homegoods', 'homes', 'homesense', 'honda', 'honeywell', 95 | 'horse', 'hospital', 'host', 'hosting', 'hot', 'hoteles', 96 | 'hotels', 'hotmail', 'house', 'how', 'hr', 'hsbc', 'ht', 'htc', 97 | 'hu', 'hughes', 'hyatt', 'hyundai', 'ibm', 'icbc', 'ice', 98 | 'icu', 'id', 'ie', 'ieee', 'ifm', 'ikano', 'il', 'im', 99 | 'imamat', 'imdb', 'immo', 'immobilien', 'in', 'inc', 100 | 'industries', 'infiniti', 'info', 'ing', 'ink', 'institute', 101 | 'insurance', 'insure', 'int', 'intel', 'international', 102 | 'intuit', 'investments', 'io', 'ipiranga', 'iq', 'ir', 'irish', 103 | 'is', 'iselect', 'ismaili', 'ist', 'istanbul', 'it', 'itau', 104 | 'itv', 'iveco', 'iwc', 'jaguar', 'java', 'jcb', 'jcp', 'je', 105 | 'jeep', 'jetzt', 'jewelry', 'jio', 'jlc', 'jll', 'jm', 'jmp', 106 | 'jnj', 'jo', 'jobs', 'joburg', 'jot', 'joy', 'jp', 'jpmorgan', 107 | 'jprs', 'juegos', 'juniper', 'kaufen', 'kddi', 'ke', 108 | 'kerryhotels', 'kerrylogistics', 'kerryproperties', 'kfh', 109 | 'kg', 'kh', 'ki', 'kia', 'kids', 'kim', 'kinder', 'kindle', 110 | 'kitchen', 'kiwi', 'km', 'kn', 'koeln', 'komatsu', 'kosher', 111 | 'kp', 'kpmg', 'kpn', 'kr', 'krd', 'kred', 'kuokgroup', 'kw', 112 | 'ky', 'kyoto', 'kz', 'la', 'lacaixa', 'ladbrokes', 113 | 'lamborghini', 'lamer', 'lancaster', 'lancia', 'lancome', 114 | 'land', 'landrover', 'lanxess', 'lasalle', 'lat', 'latino', 115 | 'latrobe', 'law', 'lawyer', 'lb', 'lc', 'lds', 'lease', 116 | 'leclerc', 'lefrak', 'legal', 'lego', 'lexus', 'lgbt', 'li', 117 | 'liaison', 'lidl', 'life', 'lifeinsurance', 'lifestyle', 118 | 'lighting', 'like', 'lilly', 'limited', 'limo', 'lincoln', 119 | 'linde', 'link', 'lipsy', 'live', 'living', 'lixil', 'lk', 120 | 'llc', 'llp', 'loan', 'loans', 'locker', 'locus', 'loft', 121 | 'lol', 'london', 'lotte', 'lotto', 'love', 'lpl', 122 | 'lplfinancial', 'lr', 'ls', 'lt', 'ltd', 'ltda', 'lu', 123 | 'lundbeck', 'lupin', 'luxe', 'luxury', 'lv', 'ly', 'ma', 124 | 'macys', 'madrid', 'maif', 'maison', 'makeup', 'man', 125 | 'management', 'mango', 'map', 'market', 'marketing', 'markets', 126 | 'marriott', 'marshalls', 'maserati', 'mattel', 'mba', 'mc', 127 | 'mcd', 'mcdonalds', 'mckinsey', 'md', 'me', 'med', 'media', 128 | 'meet', 'melbourne', 'meme', 'memorial', 'men', 'menu', 'meo', 129 | 'merckmsd', 'metlife', 'mg', 'mh', 'miami', 'microsoft', 'mil', 130 | 'mini', 'mint', 'mit', 'mitsubishi', 'mk', 'ml', 'mlb', 'mls', 131 | 'mm', 'mma', 'mn', 'mo', 'mobi', 'mobile', 'mobily', 'moda', 132 | 'moe', 'moi', 'mom', 'monash', 'money', 'monster', 'montblanc', 133 | 'mopar', 'mormon', 'mortgage', 'moscow', 'moto', 'motorcycles', 134 | 'mov', 'movie', 'movistar', 'mp', 'mq', 'mr', 'ms', 'msd', 135 | 'mt', 'mtn', 'mtpc', 'mtr', 'mu', 'museum', 'music', 'mutual', 136 | 'mv', 'mw', 'mx', 'my', 'mz', 'na', 'nab', 'nadex', 'nagoya', 137 | 'name', 'nationwide', 'natura', 'navy', 'nba', 'nc', 'ne', 138 | 'nec', 'net', 'netbank', 'netflix', 'network', 'neustar', 139 | 'new', 'newholland', 'news', 'next', 'nextdirect', 'nexus', 140 | 'nf', 'nfl', 'ng', 'ngo', 'nhk', 'ni', 'nico', 'nike', 'nikon', 141 | 'ninja', 'nissan', 'nissay', 'nl', 'no', 'nokia', 142 | 'northwesternmutual', 'norton', 'now', 'nowruz', 'nowtv', 'np', 143 | 'nr', 'nra', 'nrw', 'ntt', 'nu', 'nyc', 'nz', 'obi', 144 | 'observer', 'off', 'office', 'okinawa', 'olayan', 145 | 'olayangroup', 'oldnavy', 'ollo', 'om', 'omega', 'one', 'ong', 146 | 'onl', 'online', 'onyourside', 'ooo', 'open', 'oracle', 147 | 'orange', 'org', 'organic', 'orientexpress', 'origins', 148 | 'osaka', 'otsuka', 'ott', 'ovh', 'pa', 'page', 'pamperedchef', 149 | 'panasonic', 'panerai', 'paris', 'pars', 'partners', 'parts', 150 | 'party', 'passagens', 'pay', 'pccw', 'pe', 'pet', 'pf', 151 | 'pfizer', 'pg', 'ph', 'pharmacy', 'phd', 'philips', 'phone', 152 | 'photo', 'photography', 'photos', 'physio', 'piaget', 'pics', 153 | 'pictet', 'pictures', 'pid', 'pin', 'ping', 'pink', 'pioneer', 154 | 'pizza', 'pk', 'pl', 'place', 'play', 'playstation', 155 | 'plumbing', 'plus', 'pm', 'pn', 'pnc', 'pohl', 'poker', 156 | 'politie', 'porn', 'post', 'pr', 'pramerica', 'praxi', 'press', 157 | 'prime', 'pro', 'prod', 'productions', 'prof', 'progressive', 158 | 'promo', 'properties', 'property', 'protection', 'pru', 159 | 'prudential', 'ps', 'pt', 'pub', 'pw', 'pwc', 'py', 'qa', 160 | 'qpon', 'quebec', 'quest', 'qvc', 'racing', 'radio', 'raid', 161 | 're', 'read', 'realestate', 'realtor', 'realty', 'recipes', 162 | 'red', 'redstone', 'redumbrella', 'rehab', 'reise', 'reisen', 163 | 'reit', 'reliance', 'ren', 'rent', 'rentals', 'repair', 164 | 'report', 'republican', 'rest', 'restaurant', 'review', 165 | 'reviews', 'rexroth', 'rich', 'richardli', 'ricoh', 166 | 'rightathome', 'ril', 'rio', 'rip', 'rmit', 'ro', 'rocher', 167 | 'rocks', 'rodeo', 'rogers', 'room', 'rs', 'rsvp', 'ru', 168 | 'rugby', 'ruhr', 'run', 'rw', 'rwe', 'ryukyu', 'sa', 169 | 'saarland', 'safe', 'safety', 'sakura', 'sale', 'salon', 170 | 'samsclub', 'samsung', 'sandvik', 'sandvikcoromant', 'sanofi', 171 | 'sap', 'sapo', 'sarl', 'sas', 'save', 'saxo', 'sb', 'sbi', 172 | 'sbs', 'sc', 'sca', 'scb', 'schaeffler', 'schmidt', 173 | 'scholarships', 'school', 'schule', 'schwarz', 'science', 174 | 'scjohnson', 'scor', 'scot', 'sd', 'se', 'search', 'seat', 175 | 'secure', 'security', 'seek', 'select', 'sener', 'services', 176 | 'ses', 'seven', 'sew', 'sex', 'sexy', 'sfr', 'sg', 'sh', 177 | 'shangrila', 'sharp', 'shaw', 'shell', 'shia', 'shiksha', 178 | 'shoes', 'shop', 'shopping', 'shouji', 'show', 'showtime', 179 | 'shriram', 'si', 'silk', 'sina', 'singles', 'site', 'sj', 'sk', 180 | 'ski', 'skin', 'sky', 'skype', 'sl', 'sling', 'sm', 'smart', 181 | 'smile', 'sn', 'sncf', 'so', 'soccer', 'social', 'softbank', 182 | 'software', 'sohu', 'solar', 'solutions', 'song', 'sony', 183 | 'soy', 'spa', 'space', 'spiegel', 'sport', 'spot', 184 | 'spreadbetting', 'sr', 'srl', 'srt', 'ss', 'st', 'stada', 185 | 'staples', 'star', 'starhub', 'statebank', 'statefarm', 186 | 'statoil', 'stc', 'stcgroup', 'stockholm', 'storage', 'store', 187 | 'stream', 'studio', 'study', 'style', 'su', 'sucks', 188 | 'supplies', 'supply', 'support', 'surf', 'surgery', 'suzuki', 189 | 'sv', 'swatch', 'swiftcover', 'swiss', 'sx', 'sy', 'sydney', 190 | 'symantec', 'systems', 'sz', 'tab', 'taipei', 'talk', 'taobao', 191 | 'target', 'tatamotors', 'tatar', 'tattoo', 'tax', 'taxi', 'tc', 192 | 'tci', 'td', 'tdk', 'team', 'tech', 'technology', 'tel', 193 | 'telecity', 'telefonica', 'temasek', 'tennis', 'teva', 'tf', 194 | 'tg', 'th', 'thd', 'theater', 'theatre', 'tiaa', 'tickets', 195 | 'tienda', 'tiffany', 'tips', 'tires', 'tirol', 'tj', 'tjmaxx', 196 | 'tjx', 'tk', 'tkmaxx', 'tl', 'tm', 'tmall', 'tn', 'to', 197 | 'today', 'tokyo', 'tools', 'top', 'toray', 'toshiba', 'total', 198 | 'tours', 'town', 'toyota', 'toys', 'tr', 'trade', 'trading', 199 | 'training', 'travel', 'travelchannel', 'travelers', 200 | 'travelersinsurance', 'trust', 'trv', 'tt', 'tube', 'tui', 201 | 'tunes', 'tushu', 'tv', 'tvs', 'tw', 'tz', 'ua', 'ubank', 202 | 'ubs', 'uconnect', 'ug', 'uk', 'unicom', 'university', 'uno', 203 | 'uol', 'ups', 'us', 'uy', 'uz', 'va', 'vacations', 'vana', 204 | 'vanguard', 'vc', 've', 'vegas', 'ventures', 'verisign', 205 | 'versicherung', 'vet', 'vg', 'vi', 'viajes', 'video', 'vig', 206 | 'viking', 'villas', 'vin', 'vip', 'virgin', 'visa', 'vision', 207 | 'vista', 'vistaprint', 'viva', 'vivo', 'vlaanderen', 'vn', 208 | 'vodka', 'volkswagen', 'volvo', 'vote', 'voting', 'voto', 209 | 'voyage', 'vu', 'vuelos', 'wales', 'walmart', 'walter', 'wang', 210 | 'wanggou', 'warman', 'watch', 'watches', 'weather', 211 | 'weatherchannel', 'webcam', 'weber', 'website', 'wed', 212 | 'wedding', 'weibo', 'weir', 'wf', 'whoswho', 'wien', 'wiki', 213 | 'williamhill', 'win', 'windows', 'wine', 'winners', 'wme', 214 | 'wolterskluwer', 'woodside', 'work', 'works', 'world', 'wow', 215 | 'ws', 'wtc', 'wtf', 'xbox', 'xerox', 'xfinity', 'xihuan', 216 | 'xin', 'xn--11b4c3d', 'xn--1ck2e1b', 'xn--1qqw23a', 217 | 'xn--2scrj9c', 'xn--30rr7y', 'xn--3bst00m', 'xn--3ds443g', 218 | 'xn--3e0b707e', 'xn--3hcrj9c', 'xn--3oq18vl8pn36a', 219 | 'xn--3pxu8k', 'xn--42c2d9a', 'xn--45br5cyl', 'xn--45brj9c', 220 | 'xn--45q11c', 'xn--4dbrk0ce', 'xn--4gbrim', 'xn--54b7fta0cc', 221 | 'xn--55qw42g', 'xn--55qx5d', 'xn--5su34j936bgsg', 'xn--5tzm5g', 222 | 'xn--6frz82g', 'xn--6qq986b3xl', 'xn--80adxhks', 'xn--80ao21a', 223 | 'xn--80aqecdr1a', 'xn--80asehdb', 'xn--80aswg', 'xn--8y0a063a', 224 | 'xn--90a3ac', 'xn--90ae', 'xn--90ais', 'xn--9dbq2a', 225 | 'xn--9et52u', 'xn--9krt00a', 'xn--b4w605ferd', 226 | 'xn--bck1b9a5dre4c', 'xn--c1avg', 'xn--c2br7g', 'xn--cck2b3b', 227 | 'xn--cckwcxetd', 'xn--cg4bki', 'xn--clchc0ea0b2g2a9gcd', 228 | 'xn--czr694b', 'xn--czrs0t', 'xn--czru2d', 'xn--d1acj3b', 229 | 'xn--d1alf', 'xn--e1a4c', 'xn--eckvdtc9d', 'xn--efvy88h', 230 | 'xn--estv75g', 'xn--fct429k', 'xn--fhbei', 'xn--fiq228c5hs', 231 | 'xn--fiq64b', 'xn--fiqs8s', 'xn--fiqz9s', 'xn--fjq720a', 232 | 'xn--flw351e', 'xn--fpcrj9c3d', 'xn--fzc2c9e2c', 233 | 'xn--fzys8d69uvgm', 'xn--g2xx48c', 'xn--gckr3f0f', 234 | 'xn--gecrj9c', 'xn--gk3at1e', 'xn--h2breg3eve', 'xn--h2brj9c', 235 | 'xn--h2brj9c8c', 'xn--hxt814e', 'xn--i1b6b1a6a2e', 236 | 'xn--imr513n', 'xn--io0a7i', 'xn--j1aef', 'xn--j1amh', 237 | 'xn--j6w193g', 'xn--jlq480n2rg', 'xn--jlq61u9w7b', 238 | 'xn--jvr189m', 'xn--kcrx77d1x4a', 'xn--kprw13d', 'xn--kpry57d', 239 | 'xn--kpu716f', 'xn--kput3i', 'xn--l1acc', 'xn--lgbbat1ad8j', 240 | 'xn--mgb9awbf', 'xn--mgba3a3ejt', 'xn--mgba3a4f16a', 241 | 'xn--mgba7c0bbn0a', 'xn--mgbaakc7dvf', 'xn--mgbaam7a8h', 242 | 'xn--mgbab2bd', 'xn--mgbah1a3hjkrd', 'xn--mgbai9azgqp6j', 243 | 'xn--mgbayh7gpa', 'xn--mgbb9fbpob', 'xn--mgbbh1a', 244 | 'xn--mgbbh1a71e', 'xn--mgbc0a9azcg', 'xn--mgbca7dzdo', 245 | 'xn--mgbcpq6gpa1a', 'xn--mgberp4a5d4ar', 'xn--mgbgu82a', 246 | 'xn--mgbi4ecexp', 'xn--mgbpl2fh', 'xn--mgbt3dhd', 247 | 'xn--mgbtx2b', 'xn--mgbx4cd0ab', 'xn--mix891f', 'xn--mk1bu44c', 248 | 'xn--mxtq1m', 'xn--ngbc5azd', 'xn--ngbe9e0a', 'xn--ngbrx', 249 | 'xn--node', 'xn--nqv7f', 'xn--nqv7fs00ema', 'xn--nyqy26a', 250 | 'xn--o3cw4h', 'xn--ogbpf8fl', 'xn--otu796d', 'xn--p1acf', 251 | 'xn--p1ai', 'xn--pbt977c', 'xn--pgbs0dh', 'xn--pssy2u', 252 | 'xn--q7ce6a', 'xn--q9jyb4c', 'xn--qcka1pmc', 'xn--qxa6a', 253 | 'xn--qxam', 'xn--rhqv96g', 'xn--rovu88b', 'xn--rvc1e0am3e', 254 | 'xn--s9brj9c', 'xn--ses554g', 'xn--t60b56a', 'xn--tckwe', 255 | 'xn--tiq49xqyj', 'xn--unup4y', 'xn--vermgensberater-ctb', 256 | 'xn--vermgensberatung-pwb', 'xn--vhquv', 'xn--vuq861b', 257 | 'xn--w4r85el8fhu5dnra', 'xn--w4rs40l', 'xn--wgbh1c', 258 | 'xn--wgbl6a', 'xn--xhq521b', 'xn--xkc2al3hye2a', 259 | 'xn--xkc2dl3a5ee0h', 'xn--y9a3aq', 'xn--yfro4i67o', 260 | 'xn--ygbi2ammx', 'xn--zfr164b', 'xperia', 'xxx', 'xyz', 261 | 'yachts', 'yahoo', 'yamaxun', 'yandex', 'ye', 'yodobashi', 262 | 'yoga', 'yokohama', 'you', 'youtube', 'yt', 'yun', 'za', 263 | 'zappos', 'zara', 'zero', 'zip', 'zippo', 'zm', 'zone', 264 | 'zuerich', 'zw'}) 265 | -------------------------------------------------------------------------------- /json_importer.py: -------------------------------------------------------------------------------- 1 | """Import JSON modules with drop-in compatible API, 2 | trying modules with faster JSON parsers first: orjson, ujson, json 3 | Cf. https://github.com/commoncrawl/cc-pyspark/issues/41 4 | """ 5 | 6 | try: 7 | import orjson as json 8 | except ImportError: 9 | try: 10 | import ujson as json 11 | except ImportError: 12 | import json 13 | -------------------------------------------------------------------------------- /linkmap2parquet.py: -------------------------------------------------------------------------------- 1 | from sparkcc import CCSparkJob 2 | from pyspark.sql.types import StructType, StructField, StringType 3 | 4 | 5 | class LinkMapImportJob(CCSparkJob): 6 | """Import a map of link pairs 7 | to SparkSQL and save as Parquet""" 8 | name = "LinkMapImport" 9 | 10 | output_schema = StructType([ 11 | StructField("s", StringType(), True), 12 | StructField("t", StringType(), True) 13 | ]) 14 | 15 | def map_line(self, line): 16 | return line.split('\t') 17 | 18 | def run_job(self, session): 19 | output = None 20 | if self.args.input != '': 21 | input_data = session.sparkContext.textFile( 22 | self.args.input, 23 | minPartitions=self.args.num_input_partitions) 24 | output = input_data.map(self.map_line) 25 | 26 | df = session.createDataFrame(output, schema=self.output_schema) 27 | 28 | df.dropDuplicates() \ 29 | .coalesce(self.args.num_output_partitions) \ 30 | .sortWithinPartitions('s', 't') \ 31 | .write \ 32 | .format(self.args.output_format) \ 33 | .option("compression", self.args.output_compression) \ 34 | .saveAsTable(self.args.output) 35 | 36 | 37 | if __name__ == "__main__": 38 | job = LinkMapImportJob() 39 | job.run() 40 | -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | botocore 2 | boto3 3 | requests 4 | ujson 5 | orjson 6 | warcio 7 | 8 | # for link extraction and webgraph construction also: 9 | idna 10 | 11 | # for parsing HTML (used in cc_index_word_count.py) 12 | beautifulsoup4 13 | lxml 14 | 15 | # for HDFS support (requires environments variables JAVA_HOME and HADOOP_HOME): 16 | #pydoop 17 | 18 | # to parse WARC/WAT/WET files using FastWARC (https://pypi.org/project/FastWARC/) 19 | # cf. https://github.com/commoncrawl/cc-pyspark/issues/37 20 | #fastwarc 21 | # (tested with) 22 | #fastwarc==0.15.2 23 | 24 | # to parse HTML (used in cc_index_word_count.py) using Resiliparse (https://pypi.org/project/Resiliparse/). 25 | # Resiliparse requires compatible fastwarc version. 26 | # cf. https://github.com/commoncrawl/cc-pyspark/issues/43 27 | #Resiliparse 28 | # (tested with) 29 | #Resiliparse==0.15.2 30 | -------------------------------------------------------------------------------- /resiliparse_parser.py: -------------------------------------------------------------------------------- 1 | from resiliparse.extract.html2text import extract_plain_text 2 | from resiliparse.parse import detect_encoding 3 | from resiliparse.parse.html import HTMLTree 4 | 5 | 6 | class HTMLParser(object): 7 | """ 8 | HTML parser using Resiliparse 9 | """ 10 | 11 | def html_to_text(self, tree, **kwargs) -> str: 12 | """ 13 | Convert HTML content to plain text using Resiliparse. 14 | 15 | Returns: 16 | str: Extracted plain text with scripts and styles removed 17 | """ 18 | text = extract_plain_text(tree, **kwargs) 19 | return text 20 | 21 | def get_html_tree(self, page: bytes, encoding: str=None, **kwargs) -> HTMLTree: 22 | """ 23 | Get the HTML tree object 24 | 25 | Args: 26 | page (bytes): Raw HTML content as bytes 27 | encoding (str, optional): Specific character encoding to use. If None, auto-detection is attempted 28 | **kwargs: Additional arguments passed to extract_plain_text: 29 | Refer here https://resiliparse.chatnoir.eu/en/latest/api/extract/html2text.html#resiliparse.extract.html2text.extract_plain_text for accepted arguments. 30 | Returns: 31 | str: Extracted plain text content 32 | """ 33 | if not encoding: 34 | encoding = detect_encoding(page) 35 | tree = HTMLTree.parse_from_bytes(page, encoding, **kwargs) 36 | return tree -------------------------------------------------------------------------------- /server_count.py: -------------------------------------------------------------------------------- 1 | from sparkcc import CCSparkJob 2 | from json_importer import json 3 | 4 | 5 | class ServerCountJob(CCSparkJob): 6 | """ Count server names sent in HTTP response header 7 | (WARC and WAT is allowed as input)""" 8 | 9 | name = "CountServers" 10 | fallback_server_name = '(no server in HTTP header)' 11 | 12 | def process_record(self, record): 13 | # Notes: 14 | # - HTTP headers may include multiple "Server" headers, often indicating 15 | # a delivery cascade, e.g. nginx + idlb1 (load balancer). 16 | # We iterate over all "Server" headers in WARC records, in order to count 17 | # multiple (but unique) "Server" headers. 18 | # - WAT records store HTTP headers as JSON objects not preserving multiple 19 | # headers, see https://github.com/commoncrawl/ia-web-commons/issues/18 20 | 21 | if self.is_wat_json_record(record): 22 | # WAT (response) record 23 | record = json.loads(self.get_payload_stream(record).read()) 24 | try: 25 | payload = record['Envelope']['Payload-Metadata'] 26 | if 'HTTP-Response-Metadata' in payload: 27 | try: 28 | server_names = [] 29 | headers = payload['HTTP-Response-Metadata']['Headers'] 30 | for header in headers: 31 | if header.lower() == 'server': 32 | if isinstance(headers[header], list): 33 | for server_name in headers[header]: 34 | server_names.append(server_name.strip()) 35 | else: 36 | server_names.append(headers[header].strip()) 37 | if server_names: 38 | for server_name in server_names: 39 | if server_name != '': 40 | yield server_name, 1 41 | else: 42 | yield ServerCountJob.fallback_server_name, 1 43 | except KeyError: 44 | yield ServerCountJob.fallback_server_name, 1 45 | else: 46 | # WAT request or metadata records 47 | pass 48 | except KeyError: 49 | self.get_logger().warn("No payload metadata in WAT record for %s", 50 | self.get_warc_header(record, 'WARC-Target-URI')) 51 | 52 | elif self.is_response_record(record): 53 | # WARC response record 54 | server_names = set() 55 | for (name, value) in self.get_http_headers(record): 56 | if name.lower() == 'server': 57 | if value == '': 58 | pass 59 | elif value in server_names: 60 | self.get_logger().debug( 61 | "Not counting duplicated 'Server' header value for %s: %s", 62 | self.get_warc_header(record, 'WARC-Target-URI')) 63 | else: 64 | yield value, 1 65 | server_names.add(value) 66 | if not server_names: 67 | yield ServerCountJob.fallback_server_name, 1 68 | elif len(server_names) > 1: 69 | self.get_logger().info( 70 | "Multiple 'Server' header values for %s: %s", 71 | self.get_warc_header(record, 'WARC-Target-URI'), server_names) 72 | 73 | else: 74 | # warcinfo, request, non-WAT metadata records 75 | pass 76 | 77 | 78 | if __name__ == "__main__": 79 | job = ServerCountJob() 80 | job.run() 81 | -------------------------------------------------------------------------------- /server_count_fastwarc.py: -------------------------------------------------------------------------------- 1 | from fastwarc.warc import WarcRecordType 2 | 3 | from sparkcc_fastwarc import CCFastWarcSparkJob 4 | from server_count import ServerCountJob 5 | 6 | 7 | class ServerCountFastWarcJob(ServerCountJob, CCFastWarcSparkJob): 8 | """ Count server names sent in HTTP response header 9 | (WARC and WAT is allowed as input) using FastWARC 10 | to parse WARC files""" 11 | 12 | name = "CountServers" 13 | 14 | # process only WARC response and metadata (including WAT) records 15 | fastwarc_record_filter = WarcRecordType.metadata | WarcRecordType.response 16 | 17 | # process_record is implemented by ServerCountJob 18 | 19 | 20 | if __name__ == "__main__": 21 | job = ServerCountFastWarcJob() 22 | job.run() 23 | -------------------------------------------------------------------------------- /server_ip_address.py: -------------------------------------------------------------------------------- 1 | import ujson as json 2 | 3 | from urllib.parse import urlparse 4 | 5 | from pyspark.sql.types import StructType, StructField, StringType, LongType 6 | 7 | from sparkcc import CCSparkJob 8 | 9 | 10 | class ServerIPAddressJob(CCSparkJob): 11 | """ Collect server IP addresses from WARC response records 12 | (WARC and WAT is allowed as input)""" 13 | 14 | name = "ServerIPAddresses" 15 | 16 | output_schema = StructType([ 17 | StructField("key", StructType([ 18 | StructField("host", StringType(), True), 19 | StructField("ip", StringType(), True)]), True), 20 | StructField("cnt", LongType(), True) 21 | ]) 22 | 23 | response_no_ip_address = '(no IP address)' 24 | response_no_host = '(no host name)' 25 | 26 | def process_record(self, record): 27 | ip_address = None 28 | url = None 29 | 30 | if self.is_wat_json_record(record): 31 | # WAT (response) record 32 | record = json.loads(self.get_payload_stream(record).read()) 33 | try: 34 | warc_header = record['Envelope']['WARC-Header-Metadata'] 35 | if warc_header['WARC-Type'] != 'response': 36 | # WAT request or metadata records 37 | return 38 | if 'WARC-IP-Address' in warc_header: 39 | ip_address = warc_header['WARC-IP-Address'] 40 | url = warc_header['WARC-Target-URI'] 41 | else: 42 | # WAT metadata records 43 | return 44 | except KeyError: 45 | pass 46 | elif self.is_response_record(record): 47 | # WARC response record 48 | ip_address = self.get_warc_header(record, 'WARC-IP-Address') 49 | url = self.get_warc_header(record, 'WARC-Target-URI') 50 | else: 51 | # warcinfo, request, non-WAT metadata records 52 | return 53 | 54 | if not ip_address or ip_address == '': 55 | ip_address = ServerIPAddressJob.response_no_ip_address 56 | 57 | host_name = ServerIPAddressJob.response_no_host 58 | if url: 59 | try: 60 | host_name = urlparse(url).hostname 61 | except: 62 | pass 63 | 64 | yield (host_name, ip_address), 1 65 | 66 | 67 | if __name__ == "__main__": 68 | job = ServerIPAddressJob() 69 | job.run() 70 | -------------------------------------------------------------------------------- /sparkcc.py: -------------------------------------------------------------------------------- 1 | import argparse 2 | import json 3 | import logging 4 | import os 5 | import re 6 | 7 | from io import BytesIO 8 | from tempfile import SpooledTemporaryFile, TemporaryFile 9 | 10 | import boto3 11 | import botocore 12 | import requests 13 | 14 | from pyspark.sql import SparkSession 15 | from pyspark.sql.types import StructType, StructField, StringType, LongType 16 | 17 | from warcio.archiveiterator import ArchiveIterator 18 | from warcio.recordloader import ArchiveLoadFailed, ArcWarcRecord 19 | 20 | 21 | LOGGING_FORMAT = '%(asctime)s %(levelname)s %(name)s: %(message)s' 22 | 23 | 24 | class CCSparkJob(object): 25 | """ 26 | A simple Spark job definition to process Common Crawl data 27 | (WARC/WAT/WET files using Spark and warcio) 28 | """ 29 | 30 | name = 'CCSparkJob' 31 | 32 | output_schema = StructType([ 33 | StructField("key", StringType(), True), 34 | StructField("val", LongType(), True) 35 | ]) 36 | 37 | # description of input and output shown by --help 38 | input_descr = "Path to file listing input paths" 39 | output_descr = "Name of output table (saved in spark.sql.warehouse.dir)" 40 | 41 | # parse HTTP headers of WARC records (derived classes may override this) 42 | warc_parse_http_header = True 43 | 44 | args = None 45 | records_processed = None 46 | warc_input_processed = None 47 | warc_input_failed = None 48 | log_level = 'INFO' 49 | logging.basicConfig(level=log_level, format=LOGGING_FORMAT) 50 | 51 | num_input_partitions = 400 52 | num_output_partitions = 10 53 | 54 | # S3 client is thread-safe, cf. 55 | # https://boto3.amazonaws.com/v1/documentation/api/latest/guide/clients.html#multithreading-or-multiprocessing-with-clients) 56 | s3client = None 57 | 58 | # pattern to split a data URL (:/// or :/) 59 | data_url_pattern = re.compile('^(s3|https?|file|hdfs):(?://([^/]*))?/(.*)') 60 | 61 | 62 | def parse_arguments(self): 63 | """Returns the parsed arguments from the command line""" 64 | 65 | description = self.name 66 | if self.__doc__ is not None: 67 | description += " - " 68 | description += self.__doc__ 69 | arg_parser = argparse.ArgumentParser(prog=self.name, description=description, 70 | conflict_handler='resolve') 71 | 72 | arg_parser.add_argument("input", help=self.input_descr) 73 | arg_parser.add_argument("output", help=self.output_descr) 74 | arg_parser.add_argument("--input_base_url", 75 | help="Base URL (prefix) used if paths to WARC/WAT/WET " 76 | "files are relative paths. Used to select the " 77 | "access method: s3://commoncrawl/ (authenticated " 78 | "S3) or https://data.commoncrawl.org/ (HTTP)") 79 | arg_parser.add_argument("--num_input_partitions", type=int, 80 | default=self.num_input_partitions, 81 | help="Number of input splits/partitions, " 82 | "number of parallel tasks to process WARC " 83 | "files/records") 84 | arg_parser.add_argument("--num_output_partitions", type=int, 85 | default=self.num_output_partitions, 86 | help="Number of output partitions") 87 | arg_parser.add_argument("--output_format", default="parquet", 88 | help="Output format: parquet (default)," 89 | " orc, json, csv") 90 | arg_parser.add_argument("--output_compression", default="gzip", 91 | help="Output compression codec: None," 92 | " gzip/zlib (default), zstd, snappy, lzo, etc.") 93 | arg_parser.add_argument("--output_option", action='append', default=[], 94 | help="Additional output option pair" 95 | " to set (format-specific) output options, e.g.," 96 | " `header=true` to add a header line to CSV files." 97 | " Option name and value are split at `=` and" 98 | " multiple options can be set by passing" 99 | " `--output_option =` multiple times") 100 | 101 | arg_parser.add_argument("--local_temp_dir", default=None, 102 | help="Local temporary directory, used to" 103 | " buffer content from S3") 104 | 105 | arg_parser.add_argument("--log_level", default=self.log_level, 106 | help="Logging level") 107 | arg_parser.add_argument("--spark-profiler", action='store_true', 108 | help="Enable PySpark profiler and log" 109 | " profiling metrics if job has finished," 110 | " cf. spark.python.profile") 111 | 112 | self.add_arguments(arg_parser) 113 | args = arg_parser.parse_args() 114 | self.init_logging(args.log_level) 115 | if not self.validate_arguments(args): 116 | raise Exception("Arguments not valid") 117 | 118 | return args 119 | 120 | def add_arguments(self, parser): 121 | """Allows derived classes to add command-line arguments. 122 | Derived classes overriding this method must call 123 | super().add_arguments(parser) in order to add "register" 124 | arguments from all classes in the hierarchy.""" 125 | pass 126 | 127 | def validate_arguments(self, args): 128 | """Validate arguments. Derived classes overriding this method 129 | must call super().validate_arguments(args).""" 130 | if "orc" == args.output_format and "gzip" == args.output_compression: 131 | # gzip for Parquet, zlib for ORC 132 | args.output_compression = "zlib" 133 | return True 134 | 135 | def get_output_options(self): 136 | """Convert output options strings (opt=val) to kwargs""" 137 | return {x[0]: x[1] for x in map(lambda x: x.split('=', 1), 138 | self.args.output_option)} 139 | 140 | def init_logging(self, level=None, session=None): 141 | if level: 142 | self.log_level = level 143 | else: 144 | level = self.log_level 145 | logging.basicConfig(level=level, format=LOGGING_FORMAT) 146 | logging.getLogger(self.name).setLevel(level) 147 | if session: 148 | session.sparkContext.setLogLevel(level) 149 | 150 | def init_accumulators(self, session): 151 | """Register and initialize counters (aka. accumulators). 152 | Derived classes may use this method to add their own 153 | accumulators but must call super().init_accumulators(session) 154 | to also initialize counters from base classes.""" 155 | sc = session.sparkContext 156 | self.records_processed = sc.accumulator(0) 157 | self.warc_input_processed = sc.accumulator(0) 158 | self.warc_input_failed = sc.accumulator(0) 159 | 160 | def get_logger(self, session=None): 161 | """Get logger from SparkSession or (if None) from logging module""" 162 | if not session: 163 | try: 164 | session = SparkSession.getActiveSession() 165 | except AttributeError: 166 | pass # method available since Spark 3.0.0 167 | if session: 168 | return session._jvm.org.apache.log4j.LogManager \ 169 | .getLogger(self.name) 170 | return logging.getLogger(self.name) 171 | 172 | def run(self): 173 | """Run the job""" 174 | self.args = self.parse_arguments() 175 | 176 | builder = SparkSession.builder.appName(self.name) 177 | 178 | if self.args.spark_profiler: 179 | builder.config("spark.python.profile", "true") 180 | 181 | session = builder.getOrCreate() 182 | 183 | self.init_logging(self.args.log_level, session) 184 | self.init_accumulators(session) 185 | 186 | self.run_job(session) 187 | 188 | if self.args.spark_profiler: 189 | session.sparkContext.show_profiles() 190 | 191 | session.stop() 192 | 193 | def log_accumulator(self, session, acc, descr): 194 | """Log single counter/accumulator""" 195 | self.get_logger(session).info(descr.format(acc.value)) 196 | 197 | def log_accumulators(self, session): 198 | """Log counters/accumulators, see `init_accumulators`.""" 199 | self.log_accumulator(session, self.warc_input_processed, 200 | 'WARC/WAT/WET input files processed = {}') 201 | self.log_accumulator(session, self.warc_input_failed, 202 | 'WARC/WAT/WET input files failed = {}') 203 | self.log_accumulator(session, self.records_processed, 204 | 'WARC/WAT/WET records processed = {}') 205 | 206 | @staticmethod 207 | def reduce_by_key_func(a, b): 208 | return a + b 209 | 210 | def run_job(self, session): 211 | input_data = session.sparkContext.textFile(self.args.input, 212 | minPartitions=self.args.num_input_partitions) 213 | 214 | output = input_data.mapPartitionsWithIndex(self.process_warcs) \ 215 | .reduceByKey(self.reduce_by_key_func) 216 | 217 | session.createDataFrame(output, schema=self.output_schema) \ 218 | .coalesce(self.args.num_output_partitions) \ 219 | .write \ 220 | .format(self.args.output_format) \ 221 | .option("compression", self.args.output_compression) \ 222 | .options(**self.get_output_options()) \ 223 | .saveAsTable(self.args.output) 224 | 225 | self.log_accumulators(session) 226 | 227 | def get_s3_client(self): 228 | if not self.s3client: 229 | self.s3client = boto3.client('s3', use_ssl=False) 230 | return self.s3client 231 | 232 | def fetch_warc(self, uri, base_uri=None, offset=-1, length=-1): 233 | """Fetch WARC/WAT/WET files (or a record if offset and length are given)""" 234 | 235 | (scheme, netloc, path) = (None, None, None) 236 | uri_match = self.data_url_pattern.match(uri) 237 | if not uri_match and base_uri: 238 | # relative input URI (path) and base URI defined 239 | uri = base_uri + uri 240 | uri_match = self.data_url_pattern.match(uri) 241 | 242 | if uri_match: 243 | (scheme, netloc, path) = uri_match.groups() 244 | else: 245 | # keep local file paths as is 246 | path = uri 247 | 248 | stream = None 249 | 250 | if scheme == 's3': 251 | bucketname = netloc 252 | if not bucketname: 253 | self.get_logger().error("Invalid S3 URI: " + uri) 254 | return 255 | if not path: 256 | self.get_logger().error("Empty S3 path: " + uri) 257 | return 258 | elif path[0] == '/': 259 | # must strip leading / in S3 path 260 | path = path[1:] 261 | if offset > -1 and length > 0: 262 | rangereq = 'bytes={}-{}'.format(offset, (offset+length-1)) 263 | # Note: avoid logging too many small fetches 264 | #self.get_logger().debug('Fetching {} ({})'.format(uri, rangereq)) 265 | try: 266 | response = self.get_s3_client().get_object(Bucket=bucketname, 267 | Key=path, 268 | Range=rangereq) 269 | stream = BytesIO(response["Body"].read()) 270 | except botocore.client.ClientError as exception: 271 | self.get_logger().error( 272 | 'Failed to download: s3://{}/{} (offset: {}, length: {}) - {}' 273 | .format(bucketname, path, offset, length, exception)) 274 | self.warc_input_failed.add(1) 275 | return 276 | else: 277 | self.get_logger().info('Reading from S3 {}'.format(uri)) 278 | # download entire file using a temporary file for buffering 279 | warctemp = TemporaryFile(mode='w+b', dir=self.args.local_temp_dir) 280 | try: 281 | self.get_s3_client().download_fileobj(bucketname, path, warctemp) 282 | warctemp.seek(0) 283 | stream = warctemp 284 | except botocore.client.ClientError as exception: 285 | self.get_logger().error( 286 | 'Failed to download {}: {}'.format(uri, exception)) 287 | self.warc_input_failed.add(1) 288 | warctemp.close() 289 | 290 | elif scheme == 'http' or scheme == 'https': 291 | headers = None 292 | if offset > -1 and length > 0: 293 | headers = { 294 | "Range": "bytes={}-{}".format(offset, (offset + length - 1)) 295 | } 296 | # Note: avoid logging many small fetches 297 | #self.get_logger().debug('Fetching {} ({})'.format(uri, headers)) 298 | else: 299 | self.get_logger().info('Fetching {}'.format(uri)) 300 | response = requests.get(uri, headers=headers) 301 | 302 | if response.ok: 303 | # includes "HTTP 206 Partial Content" for range requests 304 | warctemp = SpooledTemporaryFile(max_size=2097152, 305 | mode='w+b', 306 | dir=self.args.local_temp_dir) 307 | warctemp.write(response.content) 308 | warctemp.seek(0) 309 | stream = warctemp 310 | else: 311 | self.get_logger().error( 312 | 'Failed to download {}: {}'.format(uri, response.status_code)) 313 | 314 | elif scheme == 'hdfs': 315 | try: 316 | import pydoop.hdfs as hdfs 317 | self.get_logger().error("Reading from HDFS {}".format(uri)) 318 | stream = hdfs.open(uri) 319 | except RuntimeError as exception: 320 | self.get_logger().error( 321 | 'Failed to open {}: {}'.format(uri, exception)) 322 | self.warc_input_failed.add(1) 323 | 324 | else: 325 | self.get_logger().info('Reading local file {}'.format(uri)) 326 | if scheme == 'file': 327 | # must be an absolute path 328 | uri = os.path.join('/', path) 329 | else: 330 | base_dir = os.path.abspath(os.path.dirname(__file__)) 331 | uri = os.path.join(base_dir, uri) 332 | try: 333 | stream = open(uri, 'rb') 334 | except IOError as exception: 335 | self.get_logger().error( 336 | 'Failed to open {}: {}'.format(uri, exception)) 337 | self.warc_input_failed.add(1) 338 | 339 | return stream 340 | 341 | def process_warcs(self, _id, iterator): 342 | """Process WARC/WAT/WET files, calling iterate_records(...) for each file""" 343 | for uri in iterator: 344 | self.warc_input_processed.add(1) 345 | 346 | stream = self.fetch_warc(uri, self.args.input_base_url) 347 | if not stream: 348 | continue 349 | 350 | for res in self.process_warc(uri, stream): 351 | yield res 352 | 353 | stream.close() 354 | 355 | def process_warc(self, uri, stream): 356 | """Parse a WARC (or WAT/WET file) using warcio, 357 | call iterate_records() to process the WARC records""" 358 | try: 359 | rec_iter = ArchiveIterator(stream, 360 | no_record_parse=(not self.warc_parse_http_header), 361 | arc2warc=True) 362 | for res in self.iterate_records(uri, rec_iter): 363 | yield res 364 | except ArchiveLoadFailed as exception: 365 | self.warc_input_failed.add(1) 366 | self.get_logger().error('Invalid WARC: {} - {}'.format(uri, exception)) 367 | 368 | def process_record(self, record): 369 | """Process a single WARC/WAT/WET record""" 370 | raise NotImplementedError('Processing record needs to be customized') 371 | 372 | def iterate_records(self, _warc_uri, archive_iterator): 373 | """Iterate over all WARC records. This method can be customized 374 | and allows to access also values from ArchiveIterator, namely 375 | WARC record offset and length.""" 376 | for record in archive_iterator: 377 | for res in self.process_record(record): 378 | yield res 379 | self.records_processed.add(1) 380 | # WARC record offset and length should be read after the record 381 | # has been processed, otherwise the record content is consumed 382 | # while offset and length are determined: 383 | # warc_record_offset = archive_iterator.get_record_offset() 384 | # warc_record_length = archive_iterator.get_record_length() 385 | 386 | @staticmethod 387 | def get_payload_stream(record: ArcWarcRecord): 388 | return record.content_stream() 389 | 390 | @staticmethod 391 | def get_warc_header(record: ArcWarcRecord, header: str, default: str=None): 392 | return record.rec_headers.get_header(header, default) 393 | 394 | @staticmethod 395 | def get_http_headers(record: ArcWarcRecord): 396 | return record.http_headers.headers 397 | 398 | @staticmethod 399 | def is_response_record(record: ArcWarcRecord): 400 | """Return true if WARC record is a WARC response record""" 401 | return record.rec_type == 'response' 402 | 403 | @staticmethod 404 | def is_wet_text_record(record: ArcWarcRecord): 405 | """Return true if WARC record is a WET text/plain record""" 406 | return (record.rec_type == 'conversion' and 407 | record.content_type == 'text/plain') 408 | 409 | @staticmethod 410 | def is_wat_json_record(record: ArcWarcRecord): 411 | """Return true if WARC record is a WAT record""" 412 | return (record.rec_type == 'metadata' and 413 | record.content_type == 'application/json') 414 | 415 | @staticmethod 416 | def is_html(record: ArcWarcRecord): 417 | """Return true if (detected) MIME type of a record is HTML""" 418 | html_types = ['text/html', 'application/xhtml+xml'] 419 | if (('WARC-Identified-Payload-Type' in record.rec_headers) and 420 | (record.rec_headers['WARC-Identified-Payload-Type'] in 421 | html_types)): 422 | return True 423 | content_type = record.http_headers.get_header('content-type', None) 424 | if content_type: 425 | for html_type in html_types: 426 | if html_type in content_type: 427 | return True 428 | return False 429 | 430 | 431 | class CCIndexSparkJob(CCSparkJob): 432 | """ 433 | Process the Common Crawl columnar URL index 434 | """ 435 | 436 | name = "CCIndexSparkJob" 437 | 438 | # description of input and output shown in --help 439 | input_descr = "Path to Common Crawl index table" 440 | 441 | def add_arguments(self, parser): 442 | parser.add_argument("--table", default="ccindex", 443 | help="name of the table data is loaded into" 444 | " (default: ccindex)") 445 | parser.add_argument("--query", default=None, required=True, 446 | help="SQL query to select rows (required).") 447 | parser.add_argument("--table_schema", default=None, 448 | help="JSON schema of the ccindex table," 449 | " implied from Parquet files if not provided.") 450 | 451 | def load_table(self, session, table_path, table_name): 452 | parquet_reader = session.read.format('parquet') 453 | if self.args.table_schema is not None: 454 | self.get_logger(session).info( 455 | "Reading table schema from {}".format(self.args.table_schema)) 456 | with open(self.args.table_schema, 'r') as s: 457 | schema = StructType.fromJson(json.loads(s.read())) 458 | parquet_reader = parquet_reader.schema(schema) 459 | df = parquet_reader.load(table_path) 460 | df.createOrReplaceTempView(table_name) 461 | self.get_logger(session).info( 462 | "Schema of table {}:\n{}".format(table_name, df.schema)) 463 | 464 | def execute_query(self, session, query): 465 | sqldf = session.sql(query) 466 | self.get_logger(session).info("Executing query: {}".format(query)) 467 | sqldf.explain() 468 | return sqldf 469 | 470 | def load_dataframe(self, session, partitions=-1): 471 | self.load_table(session, self.args.input, self.args.table) 472 | sqldf = self.execute_query(session, self.args.query) 473 | sqldf.persist() 474 | 475 | num_rows = sqldf.count() 476 | self.get_logger(session).info( 477 | "Number of records/rows matched by query: {}".format(num_rows)) 478 | 479 | if partitions > 0: 480 | self.get_logger(session).info( 481 | "Repartitioning data to {} partitions".format(partitions)) 482 | sqldf = sqldf.repartition(partitions) 483 | sqldf.persist() 484 | 485 | return sqldf 486 | 487 | def run_job(self, session): 488 | sqldf = self.load_dataframe(session, self.args.num_output_partitions) 489 | 490 | sqldf.write \ 491 | .format(self.args.output_format) \ 492 | .option("compression", self.args.output_compression) \ 493 | .options(**self.get_output_options()) \ 494 | .saveAsTable(self.args.output) 495 | 496 | self.log_accumulators(session) 497 | 498 | 499 | class CCIndexWarcSparkJob(CCIndexSparkJob): 500 | """ 501 | Process Common Crawl data (WARC records) found by the columnar URL index 502 | """ 503 | 504 | name = "CCIndexWarcSparkJob" 505 | 506 | input_descr = "Path to Common Crawl index table (with option `--query`)" \ 507 | " or extracted table containing WARC record coordinates" 508 | 509 | def add_arguments(self, parser): 510 | super(CCIndexWarcSparkJob, self).add_arguments(parser) 511 | agroup = parser.add_mutually_exclusive_group(required=True) 512 | agroup.add_argument("--query", default=None, 513 | help="SQL query to select rows. Note: the result " 514 | "is required to contain the columns `url', `warc" 515 | "_filename', `warc_record_offset' and `warc_record" 516 | "_length', make sure they're SELECTed. The column " 517 | "`content_charset' is optional and is utilized to " 518 | "read WARC record payloads with the right encoding.") 519 | agroup.add_argument("--csv", default=None, 520 | help="CSV file to load WARC records by filename, " 521 | "offset and length. The CSV file must have column " 522 | "headers and the input columns `url', " 523 | "`warc_filename', `warc_record_offset' and " 524 | "`warc_record_length' are mandatory, see also " 525 | "option --query.\nDeprecated, use instead " 526 | "`--input_table_format csv` together with " 527 | "`--input_table_option header=True` and " 528 | "`--input_table_option inferSchema=True`.") 529 | agroup.add_argument("--input_table_format", default=None, 530 | help="Data format of the input table to load WARC " 531 | "records by filename, offset and length. The input " 532 | "table is read from the path and is expected " 533 | "to include the columns `url', `warc_filename', " 534 | "`warc_record_offset' and `warc_record_length'. The " 535 | "input table is typically a result of a CTAS query " 536 | "(create table as). Allowed formats are: orc, " 537 | "json lines, csv, parquet and other formats " 538 | "supported by Spark.") 539 | parser.add_argument("--input_table_option", action='append', default=[], 540 | help="Additional input option when reading data from " 541 | "an input table (see `--input_table_format`). Options " 542 | "are passed to the Spark DataFrameReader.") 543 | 544 | def get_input_table_options(self): 545 | return {x[0]: x[1] for x in map(lambda x: x.split('=', 1), 546 | self.args.input_table_option)} 547 | 548 | def load_dataframe(self, session, partitions=-1): 549 | if self.args.query is not None: 550 | return super(CCIndexWarcSparkJob, self).load_dataframe(session, partitions) 551 | 552 | if self.args.csv is not None: 553 | sqldf = session.read.format("csv").option("header", True) \ 554 | .option("inferSchema", True).load(self.args.csv) 555 | elif self.args.input_table_format is not None: 556 | data_format = self.args.input_table_format 557 | reader = session.read.format(data_format) 558 | reader = reader.options(**self.get_input_table_options()) 559 | sqldf = reader.load(self.args.input) 560 | 561 | if partitions > 0: 562 | self.get_logger(session).info( 563 | "Repartitioning data to {} partitions".format(partitions)) 564 | sqldf = sqldf.repartition(partitions) 565 | 566 | sqldf.persist() 567 | 568 | return sqldf 569 | 570 | def process_record_with_row(self, record, row): 571 | """Process a single WARC record and the corresponding table row.""" 572 | if 'content_charset' in row: 573 | # pass `content_charset` forward to subclass processing WARC records 574 | record.rec_headers['WARC-Identified-Content-Charset'] = row['content_charset'] 575 | for res in self.process_record(record): 576 | yield res 577 | 578 | def fetch_process_warc_records(self, rows): 579 | """Fetch and process WARC records specified by columns warc_filename, 580 | warc_record_offset and warc_record_length in rows""" 581 | 582 | no_parse = (not self.warc_parse_http_header) 583 | 584 | for row in rows: 585 | url = row['url'] 586 | warc_path = row['warc_filename'] 587 | offset = int(row['warc_record_offset']) 588 | length = int(row['warc_record_length']) 589 | self.get_logger().debug("Fetching WARC record for {}".format(url)) 590 | record_stream = self.fetch_warc(warc_path, self.args.input_base_url, offset, length) 591 | try: 592 | for record in ArchiveIterator(record_stream, 593 | no_record_parse=no_parse): 594 | for res in self.process_record_with_row(record, row): 595 | yield res 596 | self.records_processed.add(1) 597 | except ArchiveLoadFailed as exception: 598 | self.warc_input_failed.add(1) 599 | self.get_logger().error( 600 | 'Invalid WARC record: {} ({}, offset: {}, length: {}) - {}' 601 | .format(url, warc_path, offset, length, exception)) 602 | 603 | def run_job(self, session): 604 | sqldf = self.load_dataframe(session, self.args.num_input_partitions) 605 | 606 | columns = ['url', 'warc_filename', 'warc_record_offset', 'warc_record_length'] 607 | if 'content_charset' in sqldf.columns: 608 | columns.append('content_charset') 609 | warc_recs = sqldf.select(*columns).rdd 610 | 611 | output = warc_recs.mapPartitions(self.fetch_process_warc_records) \ 612 | .reduceByKey(self.reduce_by_key_func) 613 | 614 | session.createDataFrame(output, schema=self.output_schema) \ 615 | .coalesce(self.args.num_output_partitions) \ 616 | .write \ 617 | .format(self.args.output_format) \ 618 | .option("compression", self.args.output_compression) \ 619 | .options(**self.get_output_options()) \ 620 | .saveAsTable(self.args.output) 621 | 622 | self.log_accumulators(session) 623 | -------------------------------------------------------------------------------- /sparkcc_fastwarc.py: -------------------------------------------------------------------------------- 1 | from fastwarc.warc import ArchiveIterator as FastWarcArchiveIterator 2 | from fastwarc.warc import WarcRecordType, WarcRecord 3 | from fastwarc.stream_io import FastWARCError 4 | 5 | from sparkcc import CCSparkJob 6 | 7 | 8 | class CCFastWarcSparkJob(CCSparkJob): 9 | """ 10 | Process Common Crawl WARC/WAT/WET files using Spark and FastWARC 11 | """ 12 | 13 | # allow subclasses to filter by record type 14 | fastwarc_record_filter = WarcRecordType.any_type 15 | 16 | def process_warc(self, uri, stream): 17 | """Parse a WARC (or WAT/WET file) via FastWARC""" 18 | try: 19 | rec_iter = FastWarcArchiveIterator(stream, 20 | record_types=self.fastwarc_record_filter, 21 | parse_http=self.warc_parse_http_header) 22 | for res in self.iterate_records(uri, rec_iter): 23 | yield res 24 | except FastWARCError as exception: 25 | self.warc_input_failed.add(1) 26 | self.get_logger().error( 27 | 'Invalid WARC: {} - {}'.format(uri, exception)) 28 | 29 | def iterate_records(self, _warc_uri, archive_iterator: FastWarcArchiveIterator): 30 | """Iterate over all WARC records. This method can be customized 31 | and allows to access also values from ArchiveIterator, namely 32 | WARC record offset and length.""" 33 | for record in archive_iterator: 34 | for res in self.process_record(record): 35 | yield res 36 | self.records_processed.add(1) 37 | # TODO: how to access WARC record offset and length, 38 | # cf. fastwarc/cli.py index 39 | 40 | @staticmethod 41 | def get_payload_stream(record: WarcRecord): 42 | return record.reader 43 | 44 | @staticmethod 45 | def get_warc_header(record: WarcRecord, header: str, default: str=None): 46 | return record.headers.get(header, default) 47 | 48 | @staticmethod 49 | def get_http_headers(record: WarcRecord): 50 | return record.http_headers.astuples() 51 | 52 | @staticmethod 53 | def is_response_record(record: WarcRecord): 54 | """Return true if WARC record is a WARC response record""" 55 | return record.record_type == WarcRecordType.response 56 | 57 | @staticmethod 58 | def is_wet_text_record(record: WarcRecord): 59 | """Return true if WARC record is a WET text/plain record""" 60 | return (record.record_type == WarcRecordType.conversion and 61 | record.headers.get('Content-Type') == 'text/plain') 62 | 63 | @staticmethod 64 | def is_wat_json_record(record: WarcRecord): 65 | """Return true if WARC record is a WAT record""" 66 | return (record.record_type == WarcRecordType.metadata and 67 | record.headers.get('Content-Type') == 'application/json') 68 | 69 | @staticmethod 70 | def is_html(record: WarcRecord): 71 | """Return true if (detected) MIME type of a record is HTML""" 72 | html_types = ['text/html', 'application/xhtml+xml'] 73 | if (('WARC-Identified-Payload-Type' in record.headers) and 74 | (record.headers['WARC-Identified-Payload-Type'] in html_types)): 75 | return True 76 | for (name, value) in record.http_headers.astuples(): 77 | if name.lower() == 'content-type': 78 | content_type = value.lower() 79 | for html_type in html_types: 80 | if html_type in content_type: 81 | return True 82 | return False 83 | 84 | 85 | -------------------------------------------------------------------------------- /wat_extract_links.py: -------------------------------------------------------------------------------- 1 | import idna 2 | import os 3 | import re 4 | 5 | from urllib.parse import urljoin, urlparse 6 | 7 | from pyspark.sql.types import StructType, StructField, StringType 8 | 9 | from sparkcc import CCSparkJob 10 | from json_importer import json 11 | 12 | 13 | class ExtractLinksJob(CCSparkJob): 14 | """Extract links from WAT files and redirects from WARC files 15 | and save them as pairs """ 16 | name = "ExtractLinks" 17 | 18 | output_schema = StructType([ 19 | StructField("s", StringType(), True), 20 | StructField("t", StringType(), True) 21 | ]) 22 | 23 | warc_parse_http_header = False 24 | 25 | processing_robotstxt_warc = False 26 | 27 | records_response = None 28 | records_response_wat = None 29 | records_response_warc = None 30 | records_response_robotstxt = None 31 | records_failed = None 32 | records_non_html = None 33 | records_response_redirect = None 34 | link_count = None 35 | 36 | http_redirect_pattern = re.compile(b'^HTTP\\s*/\\s*1\\.[01]\\s*30[12378]\\b') 37 | http_redirect_location_pattern = re.compile(b'^Location:\\s*(\\S+)', 38 | re.IGNORECASE) 39 | http_link_pattern = re.compile(r'<([^>]*)>') 40 | http_success_pattern = re.compile(b'^HTTP\\s*/\\s*1\\.[01]\\s*200\\b') 41 | robotstxt_warc_path_pattern = re.compile(r'.*/robotstxt/') 42 | robotstxt_sitemap_pattern = re.compile(b'^Sitemap:\\s*(\\S+)', 43 | re.IGNORECASE) 44 | url_abs_pattern = re.compile(r'^(?:https?:)?//') 45 | 46 | # Meta properties usually offering links: 47 | # 48 | html_meta_property_links = { 49 | 'og:url', 'og:image', 'og:image:secure_url', 50 | 'og:video', 'og:video:url', 'og:video:secure_url', 51 | 'twitter:url', 'twitter:image:src'} 52 | # Meta names usually offering links 53 | html_meta_links = { 54 | 'twitter:image', 'thumbnail', 'application-url', 55 | 'msapplication-starturl', 'msapplication-TileImage', 'vb_meta_bburl'} 56 | 57 | def add_arguments(self, parser): 58 | parser.add_argument("--intermediate_output", type=str, 59 | default=None, 60 | help="Intermediate output to recover job from") 61 | 62 | @staticmethod 63 | def _url_join(base, link): 64 | # TODO: efficiently join without reparsing base 65 | # TODO: canonicalize 66 | pass 67 | 68 | def iterate_records(self, warc_uri, archive_iterator): 69 | """Iterate over all WARC records and process them""" 70 | self.processing_robotstxt_warc \ 71 | = ExtractLinksJob.robotstxt_warc_path_pattern.match(warc_uri) 72 | for record in archive_iterator: 73 | for res in self.process_record(record): 74 | yield res 75 | self.records_processed.add(1) 76 | 77 | def process_record(self, record): 78 | link_count = 0 79 | if self.is_wat_json_record(record): 80 | try: 81 | wat_record = json.loads(self.get_payload_stream(record).read()) 82 | except ValueError as e: 83 | self.get_logger().error('Failed to load JSON: {}'.format(e)) 84 | self.records_failed.add(1) 85 | return 86 | warc_header = wat_record['Envelope']['WARC-Header-Metadata'] 87 | if warc_header['WARC-Type'] != 'response': 88 | # WAT request or metadata records 89 | return 90 | self.records_response.add(1) 91 | self.records_response_wat.add(1) 92 | url = warc_header['WARC-Target-URI'] 93 | for link in self.get_links(url, wat_record): 94 | link_count += 1 95 | yield link 96 | elif self.is_response_record(record): 97 | self.records_response.add(1) 98 | self.records_response_warc.add(1) 99 | stream = self.get_payload_stream(record) 100 | http_status_line = stream.readline() 101 | if (self.processing_robotstxt_warc and ExtractLinksJob 102 | .http_success_pattern.match(http_status_line)): 103 | self.records_response_robotstxt.add(1) 104 | for link in self.process_robotstxt(record, stream, 105 | http_status_line): 106 | link_count += 1 107 | yield link 108 | elif ExtractLinksJob.http_redirect_pattern.match(http_status_line): 109 | self.records_response_redirect.add(1) 110 | for link in self.process_redirect(record, stream, 111 | http_status_line): 112 | link_count += 1 113 | yield link 114 | else: 115 | return 116 | if link_count == 0: 117 | # ensure that the URL itself is a node in the graph 118 | # (every visited URL should be a node) 119 | uri = self.get_warc_header(record, 'WARC-Target-URI') 120 | for link in self.yield_link(uri, uri): 121 | link_count += 1 122 | yield link 123 | self.link_count.add(link_count) 124 | 125 | def process_redirect(self, record, stream, http_status_line): 126 | """Process redirects (HTTP status code 30[12378]) 127 | and yield redirect links""" 128 | line = stream.readline() 129 | while line: 130 | m = ExtractLinksJob.http_redirect_location_pattern.match(line) 131 | if m: 132 | redir_to = m.group(1).strip() 133 | try: 134 | redir_to = redir_to.decode('utf-8') 135 | except UnicodeError as e: 136 | self.get_logger().warning( 137 | 'URL with unknown encoding: {} - {}'.format( 138 | redir_to, e)) 139 | return 140 | redir_from = self.get_warc_header(record, 'WARC-Target-URI') 141 | for link in self.yield_link(redir_from, redir_to): 142 | yield link 143 | return 144 | elif line == b'\r\n': 145 | # end of HTTP header 146 | return 147 | line = stream.readline() 148 | 149 | def process_robotstxt(self, record, stream, http_status_line): 150 | # Robots.txt -> sitemap links are meaningful for host-level graphs, 151 | # page-level graphs usually do not contain the robots.txt as a node 152 | pass 153 | 154 | def yield_redirect(self, src, target, http_status_line): 155 | if src != target: 156 | yield src, target 157 | 158 | def extract_http_header_links(self, url, headers): 159 | """Extract links from WAT HTTP response headers""" 160 | links = [] 161 | for header in headers: 162 | header_name = header.lower() 163 | if header_name == 'content-location': 164 | if isinstance(headers[header], list): 165 | for cl in headers[header]: 166 | links.append(cl) 167 | else: 168 | links.append(headers[header]) 169 | elif header_name == 'link': 170 | if isinstance(headers[header], list): 171 | for li in headers[header]: 172 | for m in ExtractLinksJob.http_link_pattern.finditer(li): 173 | links.append(m.group(1)) 174 | else: 175 | for m in ExtractLinksJob.http_link_pattern.finditer(headers[header]): 176 | links.append(m.group(1)) 177 | return links 178 | 179 | def yield_http_header_links(self, url, headers): 180 | for l in self.extract_http_header_links(url, headers): 181 | yield url, l 182 | 183 | def yield_links(self, src_url, base_url, links, url_attr, opt_attr=None): 184 | # base_url = urlparse(base) 185 | if not base_url: 186 | base_url = src_url 187 | has_links = False 188 | for l in links: 189 | link = None 190 | if url_attr in l: 191 | link = l[url_attr] 192 | elif opt_attr in l and ExtractLinksJob.url_abs_pattern.match(l[opt_attr]): 193 | link = l[opt_attr] 194 | else: 195 | continue 196 | # lurl = _url_join(base_url, urlparse(link)).geturl() 197 | try: 198 | lurl = urljoin(base_url, link) 199 | except ValueError: 200 | continue 201 | has_links = True 202 | yield src_url, lurl 203 | if not has_links: 204 | # ensure that every page is a node in the graph 205 | # even if it has not outgoing links 206 | yield src_url, src_url 207 | 208 | def yield_link(self, src, target): 209 | yield src, target 210 | 211 | def get_links(self, url, record): 212 | try: 213 | response_meta = record['Envelope']['Payload-Metadata']['HTTP-Response-Metadata'] 214 | if 'Headers' in response_meta: 215 | # extract links from HTTP header 216 | for l in self.yield_http_header_links(url, response_meta['Headers']): 217 | yield l 218 | if 'HTML-Metadata' not in response_meta: 219 | self.records_non_html.add(1) 220 | return 221 | html_meta = response_meta['HTML-Metadata'] 222 | base = None 223 | if 'Head' in html_meta: 224 | head = html_meta['Head'] 225 | if 'Base' in head: 226 | try: 227 | base = urljoin(url, head['Base']) 228 | except ValueError: 229 | pass 230 | if 'Link' in head: 231 | # 232 | for l in self.yield_links(url, base, head['Link'], 'url'): 233 | yield l 234 | if 'Metas' in head: 235 | for m in head['Metas']: 236 | if (('property' in m and m['property'] 237 | in ExtractLinksJob.html_meta_property_links) 238 | or ('name' in m and m['name'] 239 | in ExtractLinksJob.html_meta_links) 240 | or ('content' in m 241 | and ExtractLinksJob.url_abs_pattern.match(m['content']))): 242 | for l in self.yield_links(url, base, [m], 'content'): 243 | yield l 244 | if 'Scripts' in head: 245 | for l in self.yield_links(url, base, head['Scripts'], 'url'): 246 | yield l 247 | if 'Links' in html_meta: 248 | for l in self.yield_links(url, base, html_meta['Links'], 249 | 'url', 'href'): 250 | yield l 251 | 252 | except KeyError as e: 253 | self.get_logger().error("Failed to parse record for {}: {}".format( 254 | url, e)) 255 | self.records_failed.add(1) 256 | 257 | def init_accumulators(self, session): 258 | super(ExtractLinksJob, self).init_accumulators(session) 259 | 260 | sc = session.sparkContext 261 | self.records_failed = sc.accumulator(0) 262 | self.records_non_html = sc.accumulator(0) 263 | self.records_response = sc.accumulator(0) 264 | self.records_response_wat = sc.accumulator(0) 265 | self.records_response_warc = sc.accumulator(0) 266 | self.records_response_redirect = sc.accumulator(0) 267 | self.records_response_robotstxt = sc.accumulator(0) 268 | self.link_count = sc.accumulator(0) 269 | 270 | def log_accumulators(self, session): 271 | super(ExtractLinksJob, self).log_accumulators(session) 272 | 273 | self.log_accumulator(session, self.records_response, 274 | 'response records = {}') 275 | self.log_accumulator(session, self.records_failed, 276 | 'records failed to process = {}') 277 | self.log_accumulator(session, self.records_non_html, 278 | 'records not HTML = {}') 279 | self.log_accumulator(session, self.records_response_wat, 280 | 'response records WAT = {}') 281 | self.log_accumulator(session, self.records_response_warc, 282 | 'response records WARC = {}') 283 | self.log_accumulator(session, self.records_response_redirect, 284 | 'response records redirects = {}') 285 | self.log_accumulator(session, self.records_response_robotstxt, 286 | 'response records robots.txt = {}') 287 | self.log_accumulator(session, self.link_count, 288 | 'non-unique link pairs = {}') 289 | 290 | def run_job(self, session): 291 | output = None 292 | if self.args.input != '': 293 | input_data = session.sparkContext.textFile( 294 | self.args.input, 295 | minPartitions=self.args.num_input_partitions) 296 | output = input_data.mapPartitionsWithIndex(self.process_warcs) 297 | 298 | if not self.args.intermediate_output: 299 | df = session.createDataFrame(output, schema=self.output_schema) 300 | else: 301 | if output is not None: 302 | session.createDataFrame(output, schema=self.output_schema) \ 303 | .write \ 304 | .format(self.args.output_format) \ 305 | .option("compression", self.args.output_compression) \ 306 | .saveAsTable(self.args.intermediate_output) 307 | self.log_accumulators(session.sparkContext) 308 | warehouse_dir = session.conf.get('spark.sql.warehouse.dir', 309 | 'spark-warehouse') 310 | intermediate_output = os.path.join(warehouse_dir, 311 | self.args.intermediate_output) 312 | df = session.read.parquet(intermediate_output) 313 | 314 | df.dropDuplicates() \ 315 | .coalesce(self.args.num_output_partitions) \ 316 | .sortWithinPartitions('s', 't') \ 317 | .write \ 318 | .format(self.args.output_format) \ 319 | .option("compression", self.args.output_compression) \ 320 | .saveAsTable(self.args.output) 321 | 322 | self.log_accumulators(session.sparkContext) 323 | 324 | 325 | class ExtractHostLinksJob(ExtractLinksJob): 326 | """Extract links from WAT files, redirects from WARC files, 327 | and sitemap links from robots.txt response records. 328 | Extract the host names, reverse the names (example.com -> com.example) 329 | and save the pairs .""" 330 | 331 | name = "ExtrHostLinks" 332 | output_schema = StructType([ 333 | StructField("s", StringType(), True), 334 | StructField("t", StringType(), True) 335 | ]) 336 | num_input_partitions = 32 337 | num_output_partitions = 16 338 | 339 | # match global links 340 | # - with URL scheme, more restrictive than specified in 341 | # https://tools.ietf.org/html/rfc3986#section-3.1 342 | # - or starting with // 343 | # (all other "relative" links are within the same host) 344 | global_link_pattern = re.compile(r'^(?:[a-z][a-z0-9]{1,5}:)?//', 345 | re.IGNORECASE|re.ASCII) 346 | 347 | # match IP addresses 348 | # - including IPs with leading `www.' (stripped) 349 | ip_pattern = re.compile(r'^(?:www\.)?\d{1,3}\.\d{1,3}\.\d{1,3}\.\d{1,3}\Z') 350 | 351 | # valid host names, relaxed allowing underscore, allowing also IDNAs 352 | # https://en.wikipedia.org/wiki/Hostname#Restrictions_on_valid_hostnames 353 | host_part_pattern = re.compile(r'^[a-z0-9]([a-z0-9_-]{0,61}[a-z0-9])?\Z', 354 | re.IGNORECASE|re.ASCII) 355 | 356 | # simple pattern to match many but not all host names in URLs 357 | url_parse_host_pattern = re.compile(r'^https?://([a-z0-9_.-]{2,253})(?:[/?#]|\Z)', 358 | re.IGNORECASE|re.ASCII) 359 | 360 | @staticmethod 361 | def get_surt_host(url): 362 | m = ExtractHostLinksJob.url_parse_host_pattern.match(url) 363 | if m: 364 | host = m.group(1) 365 | else: 366 | try: 367 | host = urlparse(url).hostname 368 | except Exception as e: 369 | # self.get_logger().debug("Failed to parse URL {}: {}\n".format(url, e)) 370 | return None 371 | if not host: 372 | return None 373 | host = host.strip().lower() 374 | if len(host) < 1 or len(host) > 253: 375 | return None 376 | if ExtractHostLinksJob.ip_pattern.match(host): 377 | return None 378 | parts = host.split('.') 379 | if parts[-1] == '': 380 | # trailing dot is allowed, strip it 381 | parts = parts[0:-1] 382 | if len(parts) <= 1: 383 | # do not accept single-word hosts, must be at least `domain.tld' 384 | return None 385 | if len(parts) > 2 and parts[0] == 'www': 386 | # strip leading 'www' to reduce number of "duplicate" hosts, 387 | # but leave at least 2 trailing parts (www.com is a valid domain) 388 | parts = parts[1:] 389 | for (i, part) in enumerate(parts): 390 | if len(part) > 63: 391 | return None 392 | if not ExtractHostLinksJob.host_part_pattern.match(part): 393 | try: 394 | idn = idna.encode(part).decode('ascii') 395 | except (idna.IDNAError, idna.core.InvalidCodepoint, UnicodeError, IndexError, Exception): 396 | # self.get_logger().debug("Invalid host name: {}".format(url)) 397 | return None 398 | 399 | # TODO: idna verifies the resulting string for length restrictions or invalid chars, 400 | # maybe no further verification is required: 401 | if ExtractHostLinksJob.host_part_pattern.match(idn): 402 | parts[i] = idn 403 | else: 404 | # self.get_logger().debug("Invalid host name: {}".format(url)) 405 | return None 406 | parts.reverse() 407 | return '.'.join(parts) 408 | 409 | def yield_links(self, src_url, base_url, links, url_attr, opt_attr=None, 410 | src_host=None, base_host=None): 411 | if not src_host: 412 | src_host = ExtractHostLinksJob.get_surt_host(src_url) 413 | if base_url and not base_host: 414 | base_host = ExtractHostLinksJob.get_surt_host(base_url) 415 | if base_host and not src_host: 416 | src_host = base_host 417 | if not src_host: 418 | return 419 | target_hosts = set() 420 | inner_host_links = 0 421 | for l in links: 422 | if not l: 423 | continue 424 | if url_attr in l: 425 | link = l[url_attr] 426 | elif opt_attr in l and ExtractLinksJob.url_abs_pattern.match(l[opt_attr]): 427 | link = l[opt_attr] 428 | else: 429 | continue 430 | if self.global_link_pattern.match(link): 431 | try: 432 | thost = ExtractHostLinksJob.get_surt_host(link) 433 | if not thost: 434 | pass # no host, e.g., http:///abc/, file:///C:... 435 | elif thost == src_host: 436 | pass # global link to same host 437 | else: 438 | target_hosts.add(thost) 439 | except ValueError: 440 | pass 441 | else: 442 | inner_host_links += 1 443 | for t in target_hosts: 444 | yield src_host, t 445 | if inner_host_links > 0 and base_url is not None: 446 | if base_host and base_host != src_host: 447 | # any internal link becomes an external link 448 | yield src_host, base_host 449 | 450 | def yield_link(self, src, target): 451 | src_host = ExtractHostLinksJob.get_surt_host(src) 452 | thost = ExtractHostLinksJob.get_surt_host(target) 453 | if thost and src_host: 454 | yield src_host, thost 455 | 456 | def yield_http_header_links(self, url, headers, src_host=None): 457 | links = self.extract_http_header_links(url, headers) 458 | if links: 459 | if not src_host: 460 | src_host = ExtractHostLinksJob.get_surt_host(url) 461 | if not src_host: 462 | return 463 | for link in links: 464 | host = ExtractHostLinksJob.get_surt_host(link) 465 | if host is not None and src_host != host: 466 | yield src_host, host 467 | 468 | def get_links(self, url, record): 469 | try: 470 | response_meta = record['Envelope']['Payload-Metadata']['HTTP-Response-Metadata'] 471 | src_host = ExtractHostLinksJob.get_surt_host(url) 472 | if src_host: 473 | if 'Headers' in response_meta: 474 | # extract links from HTTP header 475 | for l in self.yield_http_header_links(url, response_meta['Headers'], 476 | src_host=src_host): 477 | yield l 478 | if 'HTML-Metadata' not in response_meta: 479 | self.records_non_html.add(1) 480 | return 481 | html_meta = response_meta['HTML-Metadata'] 482 | base = None 483 | base_host = None 484 | if 'Head' in html_meta: 485 | head = html_meta['Head'] 486 | if 'Base' in head: 487 | try: 488 | base = urljoin(url, head['Base']) 489 | base_host = ExtractHostLinksJob.get_surt_host(base) 490 | except ValueError: 491 | pass 492 | if 'Link' in head: 493 | # 494 | for l in self.yield_links(url, base, head['Link'], 'url', 495 | src_host=src_host, base_host=base_host): 496 | yield l 497 | if 'Metas' in head: 498 | for m in head['Metas']: 499 | if (('property' in m and m['property'] 500 | in ExtractLinksJob.html_meta_property_links) 501 | or ('name' in m and m['name'] 502 | in ExtractLinksJob.html_meta_links) 503 | or ('content' in m 504 | and ExtractLinksJob.url_abs_pattern.match(m['content']))): 505 | for l in self.yield_links(url, base, [m], 'content', 506 | src_host=src_host, base_host=base_host): 507 | yield l 508 | if 'Scripts' in head: 509 | for l in self.yield_links(url, base, head['Scripts'], 'url', 510 | src_host=src_host, base_host=base_host): 511 | yield l 512 | if 'Links' in html_meta: 513 | for l in self.yield_links(url, base, html_meta['Links'], 514 | 'url', 'href', 515 | src_host=src_host, base_host=base_host): 516 | yield l 517 | 518 | except KeyError as e: 519 | self.get_logger().error("Failed to parse record for {}: {}".format( 520 | url, e)) 521 | self.records_failed.add(1) 522 | 523 | def process_robotstxt(self, record, stream, _http_status_line): 524 | """Process robots.txt and yield sitemap links""" 525 | line = stream.readline() 526 | while line: 527 | if line == b'\r\n': 528 | # end of HTTP header 529 | break 530 | line = stream.readline() 531 | line = stream.readline() 532 | while line: 533 | m = ExtractLinksJob.robotstxt_sitemap_pattern.match(line) 534 | if m: 535 | sitemap = m.group(1).strip() 536 | try: 537 | sitemap = sitemap.decode('utf-8') 538 | from_robotstxt = record.rec_headers.get_header('WARC-Target-URI') 539 | src_host = ExtractHostLinksJob.get_surt_host(from_robotstxt) 540 | thost = ExtractHostLinksJob.get_surt_host(sitemap) 541 | if thost and src_host and src_host != thost: 542 | yield src_host, thost 543 | except UnicodeError as e: 544 | self.get_logger().warning( 545 | 'URL with unknown encoding: {} - {}'.format( 546 | sitemap, e)) 547 | line = stream.readline() 548 | 549 | 550 | if __name__ == "__main__": 551 | # job = ExtractLinksJob() 552 | job = ExtractHostLinksJob() 553 | job.run() 554 | -------------------------------------------------------------------------------- /word_count.py: -------------------------------------------------------------------------------- 1 | import re 2 | 3 | from collections import Counter 4 | 5 | from pyspark.sql.types import StructType, StructField, StringType, LongType 6 | 7 | from sparkcc import CCSparkJob 8 | 9 | 10 | class WordCountJob(CCSparkJob): 11 | """ Word count (frequency list) from texts in Common Crawl WET files""" 12 | 13 | name = "WordCount" 14 | 15 | # output is > 16 | output_schema = StructType([ 17 | StructField("key", StringType(), True), 18 | StructField("val", StructType([ 19 | StructField("tf", LongType(), True), 20 | StructField("df", LongType(), True)]), True) 21 | ]) 22 | 23 | # simple Unicode-aware tokenization 24 | # (not suitable for CJK languages) 25 | word_pattern = re.compile(r'\w+', re.UNICODE) 26 | 27 | @staticmethod 28 | def reduce_by_key_func(a, b): 29 | # sum values of tuple 30 | return ((a[0] + b[0]), (a[1] + b[1])) 31 | 32 | def process_record(self, record): 33 | if not self.is_wet_text_record(record): 34 | return 35 | data = self.get_payload_stream(record).read().decode('utf-8') 36 | words = map(lambda w: w.lower(), 37 | WordCountJob.word_pattern.findall(data)) 38 | for word, count in Counter(words).items(): 39 | yield word, (count, 1) 40 | 41 | 42 | if __name__ == '__main__': 43 | job = WordCountJob() 44 | job.run() 45 | --------------------------------------------------------------------------------