Skip to content

Instantly share code, notes, and snippets.

@JoshRosen
Last active December 24, 2015 20:19
Show Gist options
  • Star 0 You must be signed in to star a gist
  • Fork 0 You must be signed in to fork a gist
  • Save JoshRosen/6856670 to your computer and use it in GitHub Desktop.
Save JoshRosen/6856670 to your computer and use it in GitHub Desktop.
IPython notebook for my PySpark demo at the San Francisco PyData Meetup: http://meetup.com/San-Francisco-PyData/events/142107482.
Display the source blob
Display the rendered blob
Raw
{
"metadata": {
"name": ""
},
"nbformat": 3,
"nbformat_minor": 0,
"worksheets": [
{
"cells": [
{
"cell_type": "markdown",
"metadata": {},
"source": [
"# Configuring a cluster for this demo\n",
"\n",
"This demo uses slightly newer versions of packages than were released on the Spark 0.8.0 AMI, including using a Spark-0.9.0 snapshot, using a newer version of the `spark-ec2` scripts, and using Python 2.7 and the latest version of IPython.\n",
"\n",
"I'm using a snapshot version of Spark because I wanted to include [the patch](https://github.com/apache/incubator-spark/pull/10) that fixes [SPARK-669](https://spark-project.atlassian.net/browse/SPARK-669).\n",
"\n",
"Here are some notes on how I configured my cluster. __Note__: this section is slightly incomplete; I'll complete these later.\n",
"\n",
"## Launching the Cluster\n",
"\n",
"- Launch a cluster with 20 `m1.xlarge` instances:\n",
"\n",
" ```\n",
" ./spark-ec2 -i ~/.ssh/berkeley-laptop.pem -k berkeley-laptop -s 20 -t m1.xlarge launch meetup\n",
" ```\n",
"\n",
"## Updating Spark components\n",
"\n",
"- Backup the current Spark version:\n",
"\n",
" ```\n",
" cp -r ~/spark/conf/ ~/sparkconf-backup\n",
" mv -r ~/spark/ ~/spark-old\n",
" ```\n",
" \n",
"- Clone and build Spark:\n",
"\n",
" ```\n",
" git clone https://github.com/apache/incubator-spark.git spark\n",
" cd spark\n",
" SPARK_HADOOP_VERSION=1.0.4 sbt/sbt clean assembly\n",
" rm -rf ~/spark/conf\n",
" cp -r ~/sparkconf-backup ~/spark/conf\n",
" ```\n",
" \n",
"- Deploy the updated Spark: since `copy-dir` doesn't have a `--delete` option yet, edit it and add that flag to the `rsync` command. Then, run\n",
"\n",
" ```\n",
" ~/spark-ec2/copy-dir ~/spark\n",
" ```\n",
" \n",
"- __TODO__: Update `spark-ec2`\n",
"\n",
"## Updating Python and IPython:\n",
"\n",
"```\n",
"yum install -y pssh\n",
"yum install -y python27 python27-devel\n",
"pssh -h /root/spark-ec2/slaves yum install -y python27\n",
"wget https://bitbucket.org/pypa/setuptools/raw/bootstrap/ez_setup.py -O - | python27\n",
"easy_install-2.7 pip\n",
"pip-2.7 install ipython[all]\n",
"pip-2.7 install requests numpy\n",
"yum install -y freetype-devel libpng-devel\n",
"pip-2.7 install matplotlib\n",
"```\n",
"\n",
"To have PySpark use this Python version, add the line\n",
"\n",
"```\n",
"export PYSPARK_PYTHON=python2.7\n",
"```\n",
"\n",
"to `spark-env.sh` and re-sync it across the workers.\n",
"\n",
"## Restarting Spark\n",
"\n",
"Now that we've upgraded everything, restart Spark:\n",
"\n",
"```\n",
"~/spark/bin/stop-all.sh\n",
"~/spark/bin/start-all.sh\n",
"```\n",
"\n",
"## Configuring IPython Notebook for Spark\n",
"\n",
"Follow [these instructions](http://nbviewer.ipython.org/6384491/00-Setup-IPython-PySpark.ipynb) to configure IPython notebook. Before running `ipython notebook`, make sure to `source spark-env.sh` so that its settings are used.\n",
"\n",
"## Copying the Wikipedia data from S3\n",
"\n",
"Edit `~/ephemeral-hdfs/conf/core-site.xml` to include your S3 credentials:\n",
"\n",
"Then, run `~/ephemeral-hdfs/bin/start-all.sh` to start Hadoop, then run\n",
"\n",
"```\n",
"~/ephemeral-hdfs/bin/hadoop distcp s3n://wiki-traffic/wiki-dump/text_with_title /wikitext\n",
"```\n",
"\n",
"to copy the Wikipedia dump from S3.\n",
"\n",
"## Misc. notes:\n",
"\n",
"- Reduce SPARK_MEM slightly to leave some extra memory for the OS."
]
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"IPython Notebook Configuration"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"from IPython.display import HTML\n",
"import requests\n",
"MASTER_HOSTNAME = requests.get(\"http://169.254.169.254/latest/meta-data/public-hostname\").text\n",
"%matplotlib inline"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 1
},
{
"cell_type": "heading",
"level": 2,
"metadata": {},
"source": [
"Helper Functions"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"def wiki_table(data):\n",
" from types import StringTypes\n",
" html = [\"<table>\"]\n",
" for row in data: \n",
" if isinstance(row, StringTypes):\n",
" row = [row]\n",
" html.append('<tr>')\n",
" html.append('<td><a href=\"http://en.wikipedia.org/wiki/%s\" target=\"_BLANK\">%s</a></td>' % (row[0], row[0]))\n",
" for col in row[1:]:\n",
" html.append(\"<td>%s</td>\" % str(col))\n",
" html.append('</tr>')\n",
" html.append('</table>')\n",
" return HTML(''.join(html))"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 2
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"Connecting to Spark"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"print CLUSTER_URL\n",
"\n",
"from pyspark import SparkContext\n",
"sc = SparkContext(CLUSTER_URL, 'ipython-notebook')"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"spark://ec2-174-129-88-152.compute-1.amazonaws.com:7077\n"
]
}
],
"prompt_number": 3
},
{
"cell_type": "markdown",
"metadata": {},
"source": [
"Let's test it out by creating a simple RDD"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"data = sc.parallelize(range(1000), 10)\n",
"data"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 4,
"text": [
"<pyspark.rdd.RDD at 0x384ac90>"
]
}
],
"prompt_number": 4
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"data.count()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 5,
"text": [
"1000"
]
}
],
"prompt_number": 5
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"data.take(10)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 6,
"text": [
"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]"
]
}
],
"prompt_number": 6
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"help(data)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"Help on RDD in module pyspark.rdd object:\n",
"\n",
"class RDD(__builtin__.object)\n",
" | A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.\n",
" | Represents an immutable, partitioned collection of elements that can be\n",
" | operated on in parallel.\n",
" | \n",
" | Methods defined here:\n",
" | \n",
" | __add__(self, other)\n",
" | Return the union of this RDD and another one.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 1, 2, 3])\n",
" | >>> (rdd + rdd).collect()\n",
" | [1, 1, 2, 3, 1, 1, 2, 3]\n",
" | \n",
" | __init__(self, jrdd, ctx)\n",
" | \n",
" | cache(self)\n",
" | Persist this RDD with the default storage level (C{MEMORY_ONLY}).\n",
" | \n",
" | cartesian(self, other)\n",
" | Return the Cartesian product of this RDD and another one, that is, the\n",
" | RDD of all pairs of elements C{(a, b)} where C{a} is in C{self} and\n",
" | C{b} is in C{other}.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 2])\n",
" | >>> sorted(rdd.cartesian(rdd).collect())\n",
" | [(1, 1), (1, 2), (2, 1), (2, 2)]\n",
" | \n",
" | checkpoint(self)\n",
" | Mark this RDD for checkpointing. It will be saved to a file inside the\n",
" | checkpoint directory set with L{SparkContext.setCheckpointDir()} and\n",
" | all references to its parent RDDs will be removed. This function must\n",
" | be called before any job has been executed on this RDD. It is strongly\n",
" | recommended that this RDD is persisted in memory, otherwise saving it\n",
" | on a file will require recomputation.\n",
" | \n",
" | cogroup(self, other, numPartitions=None)\n",
" | For each key k in C{self} or C{other}, return a resulting RDD that\n",
" | contains a tuple with the list of values for that key in C{self} as well\n",
" | as C{other}.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 4)])\n",
" | >>> y = sc.parallelize([(\"a\", 2)])\n",
" | >>> sorted(x.cogroup(y).collect())\n",
" | [('a', ([1], [2])), ('b', ([4], []))]\n",
" | \n",
" | collect(self)\n",
" | Return a list that contains all of the elements in this RDD.\n",
" | \n",
" | collectAsMap(self)\n",
" | Return the key-value pairs in this RDD to the master as a dictionary.\n",
" | \n",
" | >>> m = sc.parallelize([(1, 2), (3, 4)]).collectAsMap()\n",
" | >>> m[1]\n",
" | 2\n",
" | >>> m[3]\n",
" | 4\n",
" | \n",
" | combineByKey(self, createCombiner, mergeValue, mergeCombiners, numPartitions=None)\n",
" | Generic function to combine the elements for each key using a custom\n",
" | set of aggregation functions.\n",
" | \n",
" | Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a \"combined\n",
" | type\" C. Note that V and C can be different -- for example, one might\n",
" | group an RDD of type (Int, Int) into an RDD of type (Int, List[Int]).\n",
" | \n",
" | Users provide three functions:\n",
" | \n",
" | - C{createCombiner}, which turns a V into a C (e.g., creates\n",
" | a one-element list)\n",
" | - C{mergeValue}, to merge a V into a C (e.g., adds it to the end of\n",
" | a list)\n",
" | - C{mergeCombiners}, to combine two C's into a single one.\n",
" | \n",
" | In addition, users can control the partitioning of the output RDD.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 1), (\"a\", 1)])\n",
" | >>> def f(x): return x\n",
" | >>> def add(a, b): return a + str(b)\n",
" | >>> sorted(x.combineByKey(str, add, add).collect())\n",
" | [('a', '11'), ('b', '1')]\n",
" | \n",
" | count(self)\n",
" | Return the number of elements in this RDD.\n",
" | \n",
" | >>> sc.parallelize([2, 3, 4]).count()\n",
" | 3\n",
" | \n",
" | countByKey(self)\n",
" | Count the number of elements for each key, and return the result to the\n",
" | master as a dictionary.\n",
" | \n",
" | >>> rdd = sc.parallelize([(\"a\", 1), (\"b\", 1), (\"a\", 1)])\n",
" | >>> sorted(rdd.countByKey().items())\n",
" | [('a', 2), ('b', 1)]\n",
" | \n",
" | countByValue(self)\n",
" | Return the count of each unique value in this RDD as a dictionary of\n",
" | (value, count) pairs.\n",
" | \n",
" | >>> sorted(sc.parallelize([1, 2, 1, 2, 2], 2).countByValue().items())\n",
" | [(1, 2), (2, 3)]\n",
" | \n",
" | distinct(self)\n",
" | Return a new RDD containing the distinct elements in this RDD.\n",
" | \n",
" | >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect())\n",
" | [1, 2, 3]\n",
" | \n",
" | filter(self, f)\n",
" | Return a new RDD containing only the elements that satisfy a predicate.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 2, 3, 4, 5])\n",
" | >>> rdd.filter(lambda x: x % 2 == 0).collect()\n",
" | [2, 4]\n",
" | \n",
" | first(self)\n",
" | Return the first element in this RDD.\n",
" | \n",
" | >>> sc.parallelize([2, 3, 4]).first()\n",
" | 2\n",
" | \n",
" | flatMap(self, f, preservesPartitioning=False)\n",
" | Return a new RDD by first applying a function to all elements of this\n",
" | RDD, and then flattening the results.\n",
" | \n",
" | >>> rdd = sc.parallelize([2, 3, 4])\n",
" | >>> sorted(rdd.flatMap(lambda x: range(1, x)).collect())\n",
" | [1, 1, 1, 2, 2, 3]\n",
" | >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect())\n",
" | [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)]\n",
" | \n",
" | flatMapValues(self, f)\n",
" | Pass each value in the key-value pair RDD through a flatMap function\n",
" | without changing the keys; this also retains the original RDD's\n",
" | partitioning.\n",
" | \n",
" | fold(self, zeroValue, op)\n",
" | Aggregate the elements of each partition, and then the results for all\n",
" | the partitions, using a given associative function and a neutral \"zero\n",
" | value.\"\n",
" | \n",
" | The function C{op(t1, t2)} is allowed to modify C{t1} and return it\n",
" | as its result value to avoid object allocation; however, it should not\n",
" | modify C{t2}.\n",
" | \n",
" | >>> from operator import add\n",
" | >>> sc.parallelize([1, 2, 3, 4, 5]).fold(0, add)\n",
" | 15\n",
" | \n",
" | foreach(self, f)\n",
" | Applies a function to all elements of this RDD.\n",
" | \n",
" | >>> def f(x): print x\n",
" | >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f)\n",
" | \n",
" | getCheckpointFile(self)\n",
" | Gets the name of the file to which this RDD was checkpointed\n",
" | \n",
" | glom(self)\n",
" | Return an RDD created by coalescing all elements within each partition\n",
" | into a list.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 2, 3, 4], 2)\n",
" | >>> sorted(rdd.glom().collect())\n",
" | [[1, 2], [3, 4]]\n",
" | \n",
" | groupBy(self, f, numPartitions=None)\n",
" | Return an RDD of grouped items.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 1, 2, 3, 5, 8])\n",
" | >>> result = rdd.groupBy(lambda x: x % 2).collect()\n",
" | >>> sorted([(x, sorted(y)) for (x, y) in result])\n",
" | [(0, [2, 8]), (1, [1, 1, 3, 5])]\n",
" | \n",
" | groupByKey(self, numPartitions=None)\n",
" | Group the values for each key in the RDD into a single sequence.\n",
" | Hash-partitions the resulting RDD with into numPartitions partitions.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 1), (\"a\", 1)])\n",
" | >>> sorted(x.groupByKey().collect())\n",
" | [('a', [1, 1]), ('b', [1])]\n",
" | \n",
" | groupWith(self, other)\n",
" | Alias for cogroup.\n",
" | \n",
" | isCheckpointed(self)\n",
" | Return whether this RDD has been checkpointed or not\n",
" | \n",
" | join(self, other, numPartitions=None)\n",
" | Return an RDD containing all pairs of elements with matching keys in\n",
" | C{self} and C{other}.\n",
" | \n",
" | Each pair of elements will be returned as a (k, (v1, v2)) tuple, where\n",
" | (k, v1) is in C{self} and (k, v2) is in C{other}.\n",
" | \n",
" | Performs a hash join across the cluster.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 4)])\n",
" | >>> y = sc.parallelize([(\"a\", 2), (\"a\", 3)])\n",
" | >>> sorted(x.join(y).collect())\n",
" | [('a', (1, 2)), ('a', (1, 3))]\n",
" | \n",
" | keyBy(self, f)\n",
" | Creates tuples of the elements in this RDD by applying C{f}.\n",
" | \n",
" | >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x)\n",
" | >>> y = sc.parallelize(zip(range(0,5), range(0,5)))\n",
" | >>> sorted(x.cogroup(y).collect())\n",
" | [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))]\n",
" | \n",
" | leftOuterJoin(self, other, numPartitions=None)\n",
" | Perform a left outer join of C{self} and C{other}.\n",
" | \n",
" | For each element (k, v) in C{self}, the resulting RDD will either\n",
" | contain all pairs (k, (v, w)) for w in C{other}, or the pair\n",
" | (k, (v, None)) if no elements in other have key k.\n",
" | \n",
" | Hash-partitions the resulting RDD into the given number of partitions.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 4)])\n",
" | >>> y = sc.parallelize([(\"a\", 2)])\n",
" | >>> sorted(x.leftOuterJoin(y).collect())\n",
" | [('a', (1, 2)), ('b', (4, None))]\n",
" | \n",
" | map(self, f, preservesPartitioning=False)\n",
" | Return a new RDD containing the distinct elements in this RDD.\n",
" | \n",
" | mapPartitions(self, f, preservesPartitioning=False)\n",
" | Return a new RDD by applying a function to each partition of this RDD.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 2, 3, 4], 2)\n",
" | >>> def f(iterator): yield sum(iterator)\n",
" | >>> rdd.mapPartitions(f).collect()\n",
" | [3, 7]\n",
" | \n",
" | mapPartitionsWithSplit(self, f, preservesPartitioning=False)\n",
" | Return a new RDD by applying a function to each partition of this RDD,\n",
" | while tracking the index of the original partition.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 2, 3, 4], 4)\n",
" | >>> def f(splitIndex, iterator): yield splitIndex\n",
" | >>> rdd.mapPartitionsWithSplit(f).sum()\n",
" | 6\n",
" | \n",
" | mapValues(self, f)\n",
" | Pass each value in the key-value pair RDD through a map function\n",
" | without changing the keys; this also retains the original RDD's\n",
" | partitioning.\n",
" | \n",
" | mean(self)\n",
" | Compute the mean of this RDD's elements.\n",
" | \n",
" | >>> sc.parallelize([1, 2, 3]).mean()\n",
" | 2.0\n",
" | \n",
" | partitionBy(self, numPartitions, partitionFunc=<built-in function hash>)\n",
" | Return a copy of the RDD partitioned using the specified partitioner.\n",
" | \n",
" | >>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x))\n",
" | >>> sets = pairs.partitionBy(2).glom().collect()\n",
" | >>> set(sets[0]).intersection(set(sets[1]))\n",
" | set([])\n",
" | \n",
" | persist(self, storageLevel)\n",
" | Set this RDD's storage level to persist its values across operations after the first time\n",
" | it is computed. This can only be used to assign a new storage level if the RDD does not\n",
" | have a storage level set yet.\n",
" | \n",
" | pipe(self, command, env={})\n",
" | Return an RDD created by piping elements to a forked external process.\n",
" | \n",
" | >>> sc.parallelize([1, 2, 3]).pipe('cat').collect()\n",
" | ['1', '2', '3']\n",
" | \n",
" | reduce(self, f)\n",
" | Reduces the elements of this RDD using the specified commutative and\n",
" | associative binary operator.\n",
" | \n",
" | >>> from operator import add\n",
" | >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add)\n",
" | 15\n",
" | >>> sc.parallelize((2 for _ in range(10))).map(lambda x: 1).cache().reduce(add)\n",
" | 10\n",
" | \n",
" | reduceByKey(self, func, numPartitions=None)\n",
" | Merge the values for each key using an associative reduce function.\n",
" | \n",
" | This will also perform the merging locally on each mapper before\n",
" | sending results to a reducer, similarly to a \"combiner\" in MapReduce.\n",
" | \n",
" | Output will be hash-partitioned with C{numPartitions} partitions, or\n",
" | the default parallelism level if C{numPartitions} is not specified.\n",
" | \n",
" | >>> from operator import add\n",
" | >>> rdd = sc.parallelize([(\"a\", 1), (\"b\", 1), (\"a\", 1)])\n",
" | >>> sorted(rdd.reduceByKey(add).collect())\n",
" | [('a', 2), ('b', 1)]\n",
" | \n",
" | reduceByKeyLocally(self, func)\n",
" | Merge the values for each key using an associative reduce function, but\n",
" | return the results immediately to the master as a dictionary.\n",
" | \n",
" | This will also perform the merging locally on each mapper before\n",
" | sending results to a reducer, similarly to a \"combiner\" in MapReduce.\n",
" | \n",
" | >>> from operator import add\n",
" | >>> rdd = sc.parallelize([(\"a\", 1), (\"b\", 1), (\"a\", 1)])\n",
" | >>> sorted(rdd.reduceByKeyLocally(add).items())\n",
" | [('a', 2), ('b', 1)]\n",
" | \n",
" | rightOuterJoin(self, other, numPartitions=None)\n",
" | Perform a right outer join of C{self} and C{other}.\n",
" | \n",
" | For each element (k, w) in C{other}, the resulting RDD will either\n",
" | contain all pairs (k, (v, w)) for v in this, or the pair (k, (None, w))\n",
" | if no elements in C{self} have key k.\n",
" | \n",
" | Hash-partitions the resulting RDD into the given number of partitions.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 4)])\n",
" | >>> y = sc.parallelize([(\"a\", 2)])\n",
" | >>> sorted(y.rightOuterJoin(x).collect())\n",
" | [('a', (2, 1)), ('b', (None, 4))]\n",
" | \n",
" | sample(self, withReplacement, fraction, seed)\n",
" | Return a sampled subset of this RDD (relies on numpy and falls back\n",
" | on default random generator if numpy is unavailable).\n",
" | \n",
" | >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP\n",
" | [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98]\n",
" | \n",
" | sampleStdev(self)\n",
" | Compute the sample standard deviation of this RDD's elements (which corrects for bias in\n",
" | estimating the standard deviation by dividing by N-1 instead of N).\n",
" | \n",
" | >>> sc.parallelize([1, 2, 3]).sampleStdev()\n",
" | 1.0\n",
" | \n",
" | sampleVariance(self)\n",
" | Compute the sample variance of this RDD's elements (which corrects for bias in\n",
" | estimating the variance by dividing by N-1 instead of N).\n",
" | \n",
" | >>> sc.parallelize([1, 2, 3]).sampleVariance()\n",
" | 1.0\n",
" | \n",
" | saveAsTextFile(self, path)\n",
" | Save this RDD as a text file, using string representations of elements.\n",
" | \n",
" | >>> tempFile = NamedTemporaryFile(delete=True)\n",
" | >>> tempFile.close()\n",
" | >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name)\n",
" | >>> from fileinput import input\n",
" | >>> from glob import glob\n",
" | >>> ''.join(sorted(input(glob(tempFile.name + \"/part-0000*\"))))\n",
" | '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n'\n",
" | \n",
" | stats(self)\n",
" | Return a L{StatCounter} object that captures the mean, variance\n",
" | and count of the RDD's elements in one operation.\n",
" | \n",
" | stdev(self)\n",
" | Compute the standard deviation of this RDD's elements.\n",
" | \n",
" | >>> sc.parallelize([1, 2, 3]).stdev()\n",
" | 0.816...\n",
" | \n",
" | subtract(self, other, numPartitions=None)\n",
" | Return each value in C{self} that is not contained in C{other}.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 4), (\"b\", 5), (\"a\", 3)])\n",
" | >>> y = sc.parallelize([(\"a\", 3), (\"c\", None)])\n",
" | >>> sorted(x.subtract(y).collect())\n",
" | [('a', 1), ('b', 4), ('b', 5)]\n",
" | \n",
" | subtractByKey(self, other, numPartitions=None)\n",
" | Return each (key, value) pair in C{self} that has no pair with matching key\n",
" | in C{other}.\n",
" | \n",
" | >>> x = sc.parallelize([(\"a\", 1), (\"b\", 4), (\"b\", 5), (\"a\", 2)])\n",
" | >>> y = sc.parallelize([(\"a\", 3), (\"c\", None)])\n",
" | >>> sorted(x.subtractByKey(y).collect())\n",
" | [('b', 4), ('b', 5)]\n",
" | \n",
" | sum(self)\n",
" | Add up the elements in this RDD.\n",
" | \n",
" | >>> sc.parallelize([1.0, 2.0, 3.0]).sum()\n",
" | 6.0\n",
" | \n",
" | take(self, num)\n",
" | Take the first num elements of the RDD.\n",
" | \n",
" | This currently scans the partitions *one by one*, so it will be slow if\n",
" | a lot of partitions are required. In that case, use L{collect} to get\n",
" | the whole RDD instead.\n",
" | \n",
" | >>> sc.parallelize([2, 3, 4, 5, 6]).cache().take(2)\n",
" | [2, 3]\n",
" | >>> sc.parallelize([2, 3, 4, 5, 6]).take(10)\n",
" | [2, 3, 4, 5, 6]\n",
" | \n",
" | takeSample(self, withReplacement, num, seed)\n",
" | Return a fixed-size sampled subset of this RDD (currently requires numpy).\n",
" | \n",
" | >>> sc.parallelize(range(0, 10)).takeSample(True, 10, 1) #doctest: +SKIP\n",
" | [4, 2, 1, 8, 2, 7, 0, 4, 1, 4]\n",
" | \n",
" | union(self, other)\n",
" | Return the union of this RDD and another one.\n",
" | \n",
" | >>> rdd = sc.parallelize([1, 1, 2, 3])\n",
" | >>> rdd.union(rdd).collect()\n",
" | [1, 1, 2, 3, 1, 1, 2, 3]\n",
" | \n",
" | unpersist(self)\n",
" | Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.\n",
" | \n",
" | variance(self)\n",
" | Compute the variance of this RDD's elements.\n",
" | \n",
" | >>> sc.parallelize([1, 2, 3]).variance()\n",
" | 0.666...\n",
" | \n",
" | ----------------------------------------------------------------------\n",
" | Data descriptors defined here:\n",
" | \n",
" | __dict__\n",
" | dictionary for instance variables (if defined)\n",
" | \n",
" | __weakref__\n",
" | list of weak references to the object (if defined)\n",
" | \n",
" | context\n",
" | The L{SparkContext} that this RDD was created on.\n",
"\n"
]
}
],
"prompt_number": 10
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"data.map(lambda x: str(x)).take(10)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 11,
"text": [
"['0', '1', '2', '3', '4', '5', '6', '7', '8', '9']"
]
}
],
"prompt_number": 11
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"data.reduce(lambda x, y: x + y)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 12,
"text": [
"499500"
]
}
],
"prompt_number": 12
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"Loading Data from HDFS"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"wiki = sc.textFile(\"/wikitext\")"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 13
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"berkeley_pages = wiki.filter(lambda x: \"Berkeley\" in x)"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 14
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"%time berkeley_pages.count()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"CPU times: user 156 ms, sys: 12 ms, total: 168 ms\n",
"Wall time: 23.2 s\n"
]
},
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 15,
"text": [
"80535"
]
}
],
"prompt_number": 15
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"Spark's Web Interfaces"
]
},
{
"cell_type": "code",
"collapsed": true,
"input": [
"HTML(('<a href=\"http://%s:8080\" target=\"_BLANK\">Master Web UI</a>' % MASTER_HOSTNAME))"
],
"language": "python",
"metadata": {},
"outputs": [
{
"html": [
"<a href=\"http://ec2-174-129-88-152.compute-1.amazonaws.com:8080\" target=\"_BLANK\">Master Web UI</a>"
],
"metadata": {},
"output_type": "pyout",
"prompt_number": 16,
"text": [
"<IPython.core.display.HTML at 0x38587d0>"
]
}
],
"prompt_number": 16
},
{
"cell_type": "code",
"collapsed": true,
"input": [
"HTML(('<a href=\"http://%s:5080/ganglia\" target=\"_BLANK\">Ganglia UI</a>' % MASTER_HOSTNAME))"
],
"language": "python",
"metadata": {},
"outputs": []
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"Caching"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"wiki.cache()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 17,
"text": [
"<pyspark.rdd.RDD at 0x3860f90>"
]
}
],
"prompt_number": 17
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"%time berkeley_pages.count()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"CPU times: user 120 ms, sys: 56 ms, total: 176 ms\n",
"Wall time: 21 s\n"
]
},
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 18,
"text": [
"80535"
]
}
],
"prompt_number": 18
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"%time berkeley_pages.count()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"CPU times: user 100 ms, sys: 48 ms, total: 148 ms\n",
"Wall time: 9.26 s\n"
]
},
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 19,
"text": [
"80535"
]
}
],
"prompt_number": 19
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"Extracting Page Titles"
]
},
{
"cell_type": "code",
"collapsed": true,
"input": [
"berkeley_pages.first()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 20,
"text": [
"u'History_of_Cape_Colony_from_1806_to_1870\\t{{CapeColony}}\\\\n{{POV|date=December 2007}}\\\\nThe \\'\\'\\'history of Cape Colony from 1806 to 1870\\'\\'\\' spans the period of the history of [[Cape Colony]] during the [[Cape Frontier Wars]], also called the Kaffir Wars, which lasted from 1811 to 1858. The wars were fought between the [[Europe]]an [[colonist]]s and the native [[Xhosa]] who rebelled against continuing European rule. The Cape Colony was the first European colony in [[South Africa]], which was initially controlled by the [[Netherlands|Dutch]] but subsequently invaded and taken over by the [[United Kingdom|British]]. After war broke out again, a British force was sent once more to the [[Cape of Good Hope|Cape]]. After a battle in January 1806 on the shores of [[Table Bay]], the Dutch garrison of [[Cape Castle]] surrendered to the British under [[David Baird|Sir David Baird]], and in 1814, the colony was ceded outright by the Netherlands to the British crown. At that time, the colony extended to the [[mountain]]s in front of the vast [[Highveld|central plateau]], then called \"Bushmansland\", and had an area of about 194,000 [[square kilometre]]s and a population of some 60,000, of whom 27,000 were [[Whites|white]], 17,000 free [[Khoikhoi]] (Hottentots), and the rest [[slavery|slave]]s. These slaves were mostly imported black people and [[Cape Malays|Malays]].\\\\n\\\\n==First and second frontier wars==\\\\n{{Main|Xhosa Wars}}\\\\nThe first of several wars with the [[Xhosa]] had already been fought by the time that the Cape Colony had been ceded to the [[United Kingdom]]. The Xhosa that crossed the colonial frontier had been expelled from the district between the [[Sundays River]] and [[Fish River, Eastern Cape|Great Fish River]] known as the [[Zuurveld]], which became a neutral ground of sorts. For some time before 1811, the Xhosa had taken possession of the neutral ground and attacked the colonists. In order to expel them from the Zuurveld, [[John Graham (Albany)|Colonel John Graham]] took the area with a mixed-race army in December 1811, and finally the Xhosa were driven beyond the Fish River. On the site of Colonel Graham\\u2019s headquarters arose a town bearing his name: Graham\\'s Town, subsequently becoming [[Grahamstown]].\\\\n\\\\nA difficulty between the Cape Colony government and the Xhosa arose in 1817, the immediate cause of which was an attempt by the colonial authorities to enforce the restitution of some stolen cattle. On [[22 April]] [[1817]], led by a prophet-chief named [[Makana]], they attacked Graham\\u2019s Town, then held by a handful of white troops. Help arrived in time and the enemy were beaten back. It was then agreed that the land between the Fish and the [[Keiskamma River|Keiskamma]] rivers should be neutral territory.\\\\n\\\\n==1820 settlers==\\\\nThe war of 1817\\u201319 led to the first wave of [[immigration]] of English settlers of any considerable scale, an event with far-reaching consequences. The then governor, [[Lord Charles Henry Somerset|Lord Charles Somerset]], whose treaty arrangements with the Xhosa chiefs had proved untenable, desired to erect a barrier against the Xhosa by having white colonists settle in the border region. In 1820, upon the advice of Lord Somerset, parliament voted to spend [[British pound|\\xa3]]50,000 to promote migration to the Cape, prompting 4,000 British people to emigrate. These immigrants, who are now known as the [[1820 Settlers]], formed the [[Albany settlement]], later [[Port Elizabeth]], and made [[Grahamstown]] their headquarters. Intended primarily as a measure to secure the safety of the frontier, and regarded by the [[Government of the United Kingdom|British government]] chiefly as a way of finding employment for a few thousand of the unemployed in Britain. Yet, the emigration scheme accomplished something with more far reaching implications than its authors had intended. The new settlers, drawn from every part of the [[United Kingdom]] and from almost every grade of society, retained strong loyalty to Britain. In the course of time, they formed a counterpoint to the Dutch colonists. \\\\n\\\\nThe arrival of these immigrants also introduced the [[English language]] to the Cape. English language ordinances were issued for the first time in 1825, and in 1827, its use was extended to the conduct of judicial proceedings. Dutch was not, however, ousted, and the colonists became largely bilingual.\\\\n\\\\n==Dutch hostility to British rule==\\\\nAlthough the colony was prosperous, many Dutch farmers were as dissatisfied with British rule as they had been with that of the Dutch East India Company, though their grievances were not the same. In 1792, [[Moravian Church|Moravian]] [[Mission (Christian)|missions]] had been established for the benefit of the Khoikhoi, and in 1799, the [[London Missionary Society]] began to try to [[convert]] both the Khoikhoi and the Xhosa. The championship of Khoikhoi grievances by the missionaries caused much dissatisfaction among the majority of the colonists, whose conservative views temporarily prevailed, for in 1812, an ordinance was issued which gave magistrates the power to bind Khoikhoi children as apprentices under conditions little different from those of [[slavery]]. In the meantime, the movement for the [[abolition of slavery]] was gaining strength in England, and the missionaries appealed at length, from the colonists to Britain. \\\\n\\\\nAn incident, which occurred from 1815 to 1816, did much to make the Dutch frontiersmen permanently hostile to the British. A farmer named Bezuidenhout refused to obey a summons issued to him after a complaint from Khoikhoi was registered. He fired on the party sent to arrest him, and was killed by the return fire. This caused a miniature rebellion, and in its suppression five ringleaders were publicly hanged by the British at [[Slagter\\'s Nek]] where they had originally sworn to expel \"the English tyrants.\" The resentment caused by the hanging of these men was deepened by the circumstances of the execution, for the scaffold on which the rebels simultaneously were hanged broke from their united weight and the men were hanged one by one afterwards. The deeply religious Dutch frontiersmen believed the collapsing scaffold to be an [[act of God]]. An ordinance passed in 1827 abolished the old Dutch \"\\'\\'[[landdrost]]\\'\\'\" and \"\\'\\'heemraden\\'\\'\" courts, instead substituting [[resident magistrate]]s. The ordinance further stipulated that all legal proceedings be henceforth conducted in English. \\\\n\\\\nAs a result of the championing of the missionaries, a subsequent ordinance in 1828 granted equal rights with white people to the [[Khoikhoi]] and other free coloured people. Another ordinance in 1830 imposed heavy penalties for harsh treatment of slaves, and finally the [[abolitionism|emancipation]] of slaves was proclaimed in 1834. Each of these ordinances drew further ire from the Dutch farmers towards the government. Moreover, the inadequate compensation awarded to slave-owners, and the suspicions engendered by the method of payment, caused much resentment, and in 1835 the trend where farmers trekked into unknown country in order to escape from a disliked government recommenced. Emigration beyond the colonial border had in fact been continuous for 150 years, but it now took on larger proportions.\\\\n\\\\n==Third cape frontier war==\\\\n[[Image:Eastern Frontier, Cape of Good Hope, ca 1835.png|thumb|right|180px|The Eastern Frontier, ca 1835]]\\\\nOn the eastern border, further trouble arose between the government and the [[Xhosa]], towards whom the policy of the Cape government was marked by much vacillation. On [[11 December]] [[1834]], a government commando party killed a chief of high rank, incensing the Xhosa: an army of 10,000 men, led by [[Macomo]], a brother of the chief who had been killed, swept across the frontier, pillaged and burned the homesteads and killed all who resisted. Among the worst sufferers was a colony of freed Khoikhoi who, in 1829, had been settled in the [[Kat River]] valley by the British authorities. There were few available soldiers in the colony, but the governor, [[Benjamin d\\'Urban|Sir Benjamin d\\'Urban]] acted quickly and all available forces were mustered under [[Harry Smith (army)|Colonel Sir Harry Smith]], who reached Graham\\u2019s Town on [[6 January]] [[1835]], six days after news of the uprising had reached Cape Town. The British fought the Xhosa for nine months until hostilities were ended on [[17 September]] 1836 with the signing of a new peace treaty, by which all the country as far as the [[River Kei]] was acknowledged to be British, and its inhabitants declared British subjects. A site for the seat of government was selected and named [[King William\\u2019s Town]].\\\\n\\\\n==Great Trek==\\\\n[[Image:MapoftherouteoftheGreakTrek.jpg|right|180px|thumb|Map of the route of the [[Great Trek]]]]\\\\nThe British government did not approve of the actions of Sir Benjamin d\\'Urban, and the British Secretary for the Colonies, [[Lord Glenelg]], declared in a letter to the [[British monarchy|King]] that \"the great evil of the Cape Colony consists in its magnitude\" and demanded that the boundary be moved back to the [[Fish River, Eastern Cape|Fish River]]. He also eventually had d\\'Urban dismissed from office in 1837. \"The [[Kaffir (Historical usage in southern Africa)|Kaffirs]],\" in Lord Glenelg\\'s dispatch of [[26 December]], \"had an ample justification for war; they had to resent, and endeavoured justly, though impotently, to avenge a series of encroachments.\\u201d This attitude towards the Xhosa was one of the many reasons given by the [[Trek Boer]]s for leaving the Cape Colony. The [[Great Trek]], as it is called, lasted from 1836 to 1840. The trekkers (Boers), numbering around 7,000, founded communities with a [[republic]]an form of government beyond the [[Orange River|Orange]] and [[Vaal River|Vaal]] rivers, and in [[KwaZulu-Natal Province|Natal]], where they had been preceded, however, by British emigrants. From this time on, Cape Colony ceased to be the only European community in South Africa, though it was the most predominant for many years.\\\\n\\\\nConsiderable trouble was caused by the emigrant Boers on either side of the Orange River, where the Boers, the [[Basuto]]s, other native tribes, Bushmen, and [[Griqua]]s fought for superiority, while the Cape government endeavoured to protect the rights of the natives. On the advice of the [[missionary|missionaries]], who exercised great influence on all non-Dutch people, a number of the native states were recognised and subsidised by the Cape government with the objective of creating peace on the northern frontier. The first \"Treaty States\" to be recognised was [[Griqualand West]] of the Griqua people. Subsequent states were recognised between 1843 and 1844. While the northern frontier became more secure, the state of the eastern frontier was deplorable, with the government either unable or unwilling to protect farmers from the Xhosa.\\\\n\\\\nElsewhere, however, the colony was making progress. The change from slave to free labour proved to be advantageous to the farmers in the western provinces. An efficient [[education]] system, owing its inception to [[John Herschel|Sir John Herschel]], an [[astronomer]] who lived in Cape Colony from 1834 to 1838, was adopted. Road Boards were established and proved to be very effective in constructing new roads. A new stable industry, [[sheep]]raising, was added to the original set of [[wheat]]growing, [[cattle|cattle rearing]], and [[wine|wine making]]. By 1846, [[wool]] became the country\\'s most valuable export. A [[legislative council]] was established in 1835, giving the colonists a share in the government.\\\\n\\\\n==War of the Axe==\\\\nAnother war with the Xhosa, known as the [[War of the Axe]], broke out in 1846, when a Khoikhoi escort who had been [[manacle]]d to a Xhosa [[thief]] was [[murder]]ed while transporting the man to Graham\\u2019s Town to be tried for stealing an axe. A party of Xhosa attacked and killed the escort. The surrender of the murderer was refused, and war was declared in March 1846. The [[Ngqika]]s were the chief tribe engaged in the war, assisted by the Tambukies. The Xhosa were defeated on 7 June 1846 by [[Henry Somerset (British Army officer)|General Somerset]] on the [[Gwangu]], a few miles from [[Fort Peddie]]. However, the war continued until [[Sandili]], the chief of the Ngqika, surrendered. Other chiefs gradually followed this action, and by the beginning of 1848, the Xhosa had been completely subdued after twenty-one months of fighting.\\\\n\\\\n==Extension of British sovereignty==\\\\n[[Image:Sir Harry Smith.gif|thumb|right|170px|Sir Harry Smith]]\\\\nIn December 1847, or what was to be the last month of the War of the Axe, [[Harry Smith (army)|Sir Harry Smith]] reached Cape Town by boat to become the new governor of the colony. He reversed Glenelg\\'s policy soon after arrival. A proclamation he issued on [[17 December]], 1847, extended the borders of the colony northwards to the Orange river and eastward to the [[Keiskamma river]], and at a meeting of the Xhosa chiefs on [[23 December]], 1847, Sir Harry announced the annexation of the land between the Keiskamma and the [[Kei River]]s to the British crown, thus re-absorbing the territory abandoned by Lord Glenelg. The land was not, however, incorporated into the Cape Colony, but instead made a crown dependency under the name of [[British Kaffraria]]. For a time, the Xhosa accepted the new government in British Kaifaria since they were mainly left alone as the governor had other serious matters to contend with, including the assertion of British authority over the Boers beyond the Orange river, and the establishment of amicable relations with the [[Transvaal|Transvaal Boers]].\\\\n\\\\n==Convict agitation and granting of a constitution==\\\\nA crisis arose in the colony over a proposal to make the Cape Colony a [[convict|convict station]]. A circular written in 1848 by the third [[Earl Grey]], then colonial secretary, was sent to the governor of the Cape, as well as other colonial governors, asking them to ascertain the feelings of the colonists regarding the reception of a certain class of convicts. The Earl intended to send [[Ireland|Irish]] peasants who had been driven to crime by the [[Irish Potato Famine|famine]] of 1845 to South Africa. Due to a misunderstanding, a boat named the \\'\\'Neptune\\'\\' was sent to the Cape Colony before the colonists\\' opinion had been received. The boat had 289 convicts on board, among whom was the famous Irish rebel [[John Mitchel]], and his colleagues. When the news that this vessel was on her way reached the Cape, people became violently excited and established an anti-convict association whose members bound themselves to cease from all interaction of any kind with persons in any way associated \"with the landing, supplying or employing convicts\". Sir Harry Smith, confronted with violent public agitation, agreed not to allow the convicts to land when the \\'\\'Neptune\\'\\' arrived in Simon\\'s Bay on [[19 September]] [[1849]], but to keep them on board the ship until he received orders to send them elsewhere. When the home government became aware of the state of affairs, orders were sent directing the \\'\\'Neptune\\'\\' to proceed to [[Tasmania]], and it did so after staying in Simon\\u2019s Bay for five months. The agitation did not fade away without further achievements, as it led to another movement that intended to obtain a free, representative government for the colony. The British government granted this concession, which had been previously promised by Lord Grey, and a constitution was established in 1854 of almost unprecedented liberality.\\\\n\\\\n==Eighth frontier war of 1850-1853==\\\\nThe anti-convict move had scarcely ended when the colony was once again involved in a war. The Xhosa bitterly resented their loss of independence, and had secretly prepared to renew their struggle ever since the last war. Sir Harry Smith, informed of the increasingly threatening attitude of the natives, went to the border region and summoned Sandili and the other chiefs for a meeting. Sandili refused obedience, after which the governor declared him deposed from his chieftanship at an assembly of other chiefs in October 1850, and appointed an English magistrate named Mr Brownlee to be temporary chief of the Ngqika tribe. It seems that the governor believed that he would be able to prevent a war and that Sandili could be arrested without armed resistance. [[George Mackinnon|Colonel George Mackinnon]], who had been sent out with a small army with the goal of securing the chief, was attacked on [[24 December]], 1850, in a narrow gorge by a large number of Xhosa, and compelled to retreat after some loss of men. This small battle prompted a general rising among the whole Ngqika tribe. Settlers in military villages that had been established along the border, were caught in a surprise attack after they had gathered to celebrate [[Christmas|Christmas day]]. Many of them were killed, and their houses set on fire. \\\\n\\\\nOther setbacks followed in quick succession. The greater part of the Xhosa police deserted, many of them leaving with their arms. Emboldened by their initial success, the Xhosa surrounded and attacked [[Fort Cox]] with immense force, where the governor was stationed with a small number of soldiers. More than one unsuccessful attempt was made to kill Sir Harry, and he needed to find a way to escape. At the head of 150 mounted riflemen, accompanied by Colonel Mackinnon, he galloped out of the fort, and rode to King William\\u2019s Town through heavy enemy fire \\u2014 a distance of 12 miles (19 km). \\\\n\\\\nMeanwhile, a new enemy appeared. Some 900 of the Kat river Khoikhoi, who had in former wars been firm allies of the British, joined their former enemies: the Xhosa. They were not without justification. They complained that while serving as soldiers in former wars \\u2014 the Cape Mounted Rifles consisted largely of Khoikhois \\u2014 they had not received the same treatment as others serving in defence of the colony, that they got no compensation for the losses they had sustained, and that they were in various ways made to feel they were a wronged and injured race. A secret alliance was formed with the Xhosa to take up arms in order to remove the Europeans and establish a Khoikhoi republic. Within a fortnight of the attack on Colonel Mackinnon, the Kat river Khoikhoi were also in arms. Their revolt was followed by that of the Khoikhoi at other missionary stations, and some of the Khoikhoi of the Cape Mounted Rifles followed their example, including some of the very men who had escorted the governor from Fort Cox. But many of the Khoikhoi remained loyal, and the [[Fingo]] likewise sided with the British.\\\\n\\\\nAfter the confusion caused by the surprise attack had subsided, Sir Harry Smith and his force turned the tide of war against the Xhosa. The [[Amatola Mountains]] were stormed, and [[Sarhili]], the highest ranking chief, who had been secretly assisting the Ngqika all along, was severely punished. In April 1852, Sir Harry Smith was recalled by Earl Grey, who accused him \\u2014 unjustly, in the opinion of the Duke of Wellington \\u2014 of a want of energy and judgement in conducting the war; he was succeeded by Lieutenant-General Cathcart. Sarhili was again attacked and reduced to submission. The Amatolas were finally cleared of Xhosa, and small forts were erected to prevent their reoccupation. \\\\n\\\\nThe British commanders were hampered throughout by their insufficient equipment, and it was not until March 1853 that the largest of the Frontier wars was brought to an end after the loss several hundred British soldiers. Shortly afterwards, British Kaffraria was made a [[crown colony]]. The Khoikhoi settlement at Kat River remained, but the Khoikhoi power within the colony was crushed.\\\\n\\\\n==Xhosa cattle-killing movement and famine==<!-- This section is linked from [[Cattle-killing movement]] -->\\\\nThe Xhosa tribes gave the colony few problems after the war. This was due, in large measure, to an extraordinary [[delusion]] which arose among the Xhosa in 1856, and led in 1857 to the death of some 50,000 people. This incident is one of the most remarkable instances of misplaced faith recorded in history. The Xhosa had not accepted their defeat in 1853 as decisive and were preparing to renew their struggle with the Europeans. \\\\n\\\\nIn 1854, a disease spread through the cattle of the Xhosa. It was believed to have spread from cattle owned by the Settlers. Widespread cattle deaths resulted, and the Xhosa believed that the deaths were caused by ubuthi, or witchcraft. In April, 1856 two girls one being nongqawuse went to scare birds out of the fields. When she returned, she told her uncle Mhlakaza that she had met three spirits at the bushes, and that they had told her that all cattle should be slaughtered, and their crops destroyed. On the day following the destruction, the dead Xhosa would return and help expel the whites. The ancestors would bring cattle with them to replace those that had been killed. Mhlakaza believed the prophecy, and repeated it to the chief [[Sarhili]].\\\\n\\\\nSarhili ordered the commands of the spirits to be obeyed. At first, the Xhosa were ordered to destroy their fat cattle. Nongqawuse, standing in the river where the spirits had first appeared, heard unearthly noises, interpreted by her uncle as orders to kill more and more cattle. At length, the spirits commanded that not an animal of all their herds was to remain alive, and every grain of corn was to be destroyed. If that were done, on a given date, myriads of cattle more beautiful than those destroyed would issue from the earth, while great fields of corn, ripe and ready for harvest, would instantly appear. The dead would rise, trouble and sickness vanish, and youth and beauty come to all alike. Unbelievers and the hated white man would on that day perish. \\\\n\\\\nThe people heard and obeyed. Sarhili is believed by many people to have been the instigator of the prophecies. Certainly some of the principal chiefs believed that they were acting simply in preparation for a last struggle with the Europeans, their plan being to throw the whole Xhosa nation fully armed and famished upon the colony. Belief in the prophecy was bolstered by the death of Lieutenant-General [[George Cathcart|Cathcart]] in the [[Crimean War]] in 1854. His death was interpreted as being due to intervention by the ancestors.\\\\n\\\\nThere were those who neither believed the predictions nor looked for success in war, but destroyed their last particle of food in unquestioning obedience to their chief\\u2019s command. Either in faith that reached the [[sublime (philosophy)|sublime]], or in obedience equally great, vast numbers of the people acted. Great [[kraal]]s were also prepared for the promised cattle, and huge skin sacks to hold the milk that was soon to be more plentiful than water. At length the day dawned which, according to the prophecies, was to usher in the terrestrial paradise. The sun rose and sank, but the expected miracle did not come to pass. The chiefs who had planned to hurl the famished warriors upon the colony had committed an incredible blunder in neglecting to call the nation together under pretext of witnessing the resurrection. They realised their error too late, and attempted to fix the situation by changing the resurrection to another day, but blank despair had taken the place of hope and faith, and it was only as starving supplicants that the Xhosa sought the British. \\\\n\\\\nSir George Grey, governor of the Cape at the time ordered the European settlers not to help the Xhosa unless they entered labour contracts with the settlers who owned land in the area. In their extreme famine, many of the Xhosa turned to [[cannibalism]], and one instance of parents eating their own child is authenticated. Among the survivors was the girl Nongqawuse; however, her uncle perished. A vivid narrative of the whole incident is found in G. M. Theal\\u2019s \\'\\'History and Geography of South Africa\\'\\' (3rd edition, London, 1878). The depopulated country was afterwards peopled by European settlers, among whom were members of the German legion which had served with the British army in the [[Crimea]], and some, 2000 industrious North German emigrants, who proved a valuable acquisition to the colony.\\\\n\\\\nHistorians now view this movement as a [[millennialist]] response both directly to a lung disease spreading among Xhosa cattle at the time, and less directly to the stress to Xhosa society caused by the continuing loss of their territory and autonomy. At least one historian has also suggested that it can be seen as a rebellion against the upper classes of Xhosa society, which used cattle as a means of consolidating wealth and political power, and which had lost respect as they failed to hold back white expansion.\\\\n\\\\n==Sir George Grey\\u2019s governorship==\\\\n<!-- Deleted image removed: [[Image:SirGeorgeGrey.jpg|thumb|right|180px|Sir George Grey]] -->\\\\n[[George Edward Grey|Sir George Grey]] became governor of the Cape Colony in 1854, and the development of the colony owes much to his administration. In his opinion, policy imposed upon the colony by the home government\\'s policy of not governing beyond the Orange River was mistaken, and in 1858 he proposed a scheme for a [[confederation]] that would include all of South Africa, however it was rejected by Britain. Sir George kept open a British road through [[Bechuanaland]] to the far interior, gaining the support of the missionaries Moffat and [[David Livingstone]]. Sir George also attempted for the first time, missionary effort apart, to educate the Xhosa and to firmly establish British authority among them, which the self-destruction of the Xhosa rendered easy. Beyond the Kei River, the natives were left to their own devices.\\\\n\\\\nSir George Grey left the Cape in 1861. During his governorship the resources of the colony had increased with the opening of the [[copper]] mines in [[Little Namaqualand]], the [[mohair]] wool industry had been established and Natal made a separate colony. The opening, in November 1863, of the [[railway]] from [[Cape Town]] to [[Wellington, South Africa|Wellington]], and the construction in 1860 of the great breakwater in [[Table Bay]], long needed on that perilous coast, marked the beginning in the colony of [[public works]] on a large scale. They were the more-or-less direct result of the granting to the colony of a large share in its own government. \\\\n\\\\nThe province of British Kaffraria was incorporated into the colony in 1865, under the title of the Electoral Divisions of King William\\u2019s Town and [[East London, South Africa|East London]]. The transfer was marked by the removal of the prohibition of the sale of [[alcohol]]ic beverages to the natives, and the free trade in intoxicants which followed had most deplorable results among the Xhosa tribes. A severe drought, affecting almost the entire colony for several years, caused great economic depression, and many farmers suffered severely. It was at this period in 1869 that [[ostrich]]-farming was successfully established as a separate [[industry]].\\\\n\\\\nWhether by or against the wish of the home government, the limits of British authority continued to extend. The [[Basotho]], who dwelt in the upper valleys of the Orange River, had subsisted under a semi-protectorate of the British government from 1843 to 1854; but having been left to their own resources on the abandonment of the Orange sovereignty, they fell into a long exhaustive warfare with the Boers of the [[Orange Free State]]. On the urgent petition of their chief [[Moshesh]], they were proclaimed British subjects in 1868, and their territory became part of the Cape Colony in 1871 (see [[Basutoland]]). In the same year, the southeastern part of [[Bechuanaland]] was annexed to Britain under the title of Griqualand West. This annexation was a consequence of the discovery there of rich [[diamond]] mines, an event which was destined to have far-reaching results.\\\\n\\\\n==References==\\\\n* \\'\\'The Migant Farmer in the History of the Cape Colony\\'\\'.P.J. Van Der Merwe, Roger B. Beck. [[Ohio University Press]]. [[1 January]] [[1995]]. 333 pages. ISBN 0-8214-1090-3.\\\\n* \\'\\'History of the Boers in South Africa; Or, the Wanderings and Wars of the Emigrant Farmers from Their Leaving the Cape Colony to the Acknowledgment of Their Independence by Great Britain\\'\\'. George McCall Theal. Greenwood Press. [[28 February]] [[1970]]. 392 pages. ISBN 0-8371-1661-9.\\\\n* \\'\\'Status and Respectability in the Cape Colony, 1750\\u20131870 : A Tragedy of Manners\\'\\'. Robert Ross, David Anderson. [[Cambridge University Press]]. [[1 July]] [[1999]]. 220 pages. ISBN 0-521-62122-4.\\\\n* \\'\\'The War of the Axe, 1847: Correspondence between the governor of the Cape Colony, Sir Henry Pottinger, and the commander of the British forces at the Cape, Sir George Berkeley, and others\\'\\'. Basil Alexander Le Cordeur. Brenthurst Press. 1981. 287 pages. ISBN 0-909079-14-5.\\\\n* \\'\\'Blood Ground: Colonialism, Missions, and the Contest for Christianity in the Cape Colony and Britain, 1799\\u20131853\\'\\'. Elizabeth Elbourne. McGill-Queen\\'s University Press. December 2002. 560 pages. ISBN 0-7735-2229-8.\\\\n* \\'\\'Recession and its aftermath: The Cape Colony in the eighteen eighties\\'\\'. Alan Mabin. University of the Witwatersrand, African Studies Institute. 1983. 27 pages. ASIN B0007B2MXA.\\\\n\\\\n==External links==\\\\n* [http://www.encyclopedia.com/html/section/CapeProv_History.asp Cape Colony History on Encyclopedia.com]\\\\n* [http://www.britannica.com/eb/article?tocId=44059 Encyclop\\xe6dia Britannica Cape Colony]\\\\n\\\\n[[Category:History of colonialism]]\\\\n[[Category:History of South Africa]]\\\\n[[Category:19th century in Africa]] \\\\n\\\\n[[af:Geskiedenis van die Kaapkolonie vanaf 1806 tot 1870]]\\\\n[[de:Geschichte der Kapkolonie (1806\\u20131870)]]'"
]
}
],
"prompt_number": 20
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"for title in berkeley_pages.map(lambda x: x.split(\"\\t\")[0]).take(20):\n",
" print title"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"History_of_Cape_Colony_from_1806_to_1870\n",
"Cattle-killing_movement\n",
"Outlanders_(anime)\n",
"Outlanders_(anime\n",
"Outlanders_(manga)\n",
"Komodo_dragon\n",
"Varanus_komodoensis\n",
"Kimodo_Dragon\n",
"Comodo_dragon\n",
"Komodo_monitor\n",
"Komodo_Monitor\n",
"Komodo_Island_Monitor\n",
"Komodo_dragons\n",
"Kimodo_dragon\n",
"Komodos\n",
"Komodo_Dragon\n",
"Buaja_durat\n",
"Biawak_raksasa\n",
"Kodomo_dragon\n",
"Komodo_dragon_fact_sheet\n"
]
}
],
"prompt_number": 21
},
{
"cell_type": "heading",
"level": 1,
"metadata": {},
"source": [
"Which article has the most incoming links?"
]
},
{
"cell_type": "heading",
"level": 3,
"metadata": {},
"source": [
"Extracting Outgoing Links"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"import re\n",
"def extract_links(raw_text):\n",
" for link in re.findall(\"\\[\\[([^\\]]*)\\]\\]\", raw_text.split(\"\\t\")[1]):\n",
" splits = link.split(\"|\")\n",
" if not splits[0].startswith(\"Image:\"):\n",
" yield splits[-1]"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 22
},
{
"cell_type": "heading",
"level": 3,
"metadata": {},
"source": [
"Counting In-Links"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"linked_pages = wiki.flatMap(extract_links)"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 23
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"wiki_table(linked_pages.take(20))"
],
"language": "python",
"metadata": {},
"outputs": [
{
"html": [
"<table><tr><td><a href=\"http://en.wikipedia.org/wiki/Saudi Arabia\" target=\"_BLANK\">Saudi Arabia</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Al Qaeda in Saudi Arabia\" target=\"_BLANK\">Al Qaeda in Saudi Arabia</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Saudi Gazette\" target=\"_BLANK\">Saudi Gazette</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/most wanted list\" target=\"_BLANK\">most wanted list</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Category:Living people\" target=\"_BLANK\">Category:Living people</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/BTS Skytrain\" target=\"_BLANK\">BTS Skytrain</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/BTS skytrain\" target=\"_BLANK\">BTS skytrain</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Watthana\" target=\"_BLANK\">Watthana</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Khlong Toei\" target=\"_BLANK\">Khlong Toei</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Bangkok\" target=\"_BLANK\">Bangkok</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Thailand\" target=\"_BLANK\">Thailand</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sukhumvit Road\" target=\"_BLANK\">Sukhumvit Road</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Pridi Banomyong Road\" target=\"_BLANK\">Pridi Banomyong Road</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Soi\" target=\"_BLANK\">Soi</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Bangkok Skytrain\" target=\"_BLANK\">Bangkok Skytrain</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/th:\u0e2a\u0e16\u0e32\u0e19\u0e35\u0e1e\u0e23\u0e30\u0e42\u0e02\u0e19\u0e07\" target=\"_BLANK\">th:\u0e2a\u0e16\u0e32\u0e19\u0e35\u0e1e\u0e23\u0e30\u0e42\u0e02\u0e19\u0e07</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/ja:\u30d7\u30e9\u30ab\u30ce\u30f3\u99c5\" target=\"_BLANK\">ja:\u30d7\u30e9\u30ab\u30ce\u30f3\u99c5</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/zh:\u62cd\u5d11\u5d19\u8eca\u7ad9\" target=\"_BLANK\">zh:\u62cd\u5d11\u5d19\u8eca\u7ad9</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Category:Bangkok skytrain station\" target=\"_BLANK\">Category:Bangkok skytrain station</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Black Country\" target=\"_BLANK\">Black Country</a></td></tr></table>"
],
"metadata": {},
"output_type": "pyout",
"prompt_number": 24,
"text": [
"<IPython.core.display.HTML at 0x391ced0>"
]
}
],
"prompt_number": 24
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"linked_pages.count()"
],
"language": "python",
"metadata": {},
"outputs": []
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"number_of_inlinks = linked_pages.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y).cache()"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 25
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"%time inlink_frequencies = number_of_inlinks.map(lambda x: x[1]).countByValue()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"CPU times: user 196 ms, sys: 52 ms, total: 248 ms\n",
"Wall time: 48.1 s\n"
]
}
],
"prompt_number": 26
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"import pylab\n",
"pylab.hist(inlink_frequencies.keys(), bins=range(50), weights=inlink_frequencies.values());"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "display_data",
"png": "iVBORw0KGgoAAAANSUhEUgAAAZEAAAEACAYAAAByG0uxAAAABHNCSVQICAgIfAhkiAAAAAlwSFlz\nAAALEgAACxIB0t1+/AAAIABJREFUeJzt3X1MVGe+B/DvAG13E9ZeyzC0O2eyRG6rw2GA40UFu22w\nSpDxBe5aM6QLe1vbpGlKYc3NTf9obgvrrY21lbjpZnM3d3V1MdkaX9qmM51A0s62uUZWGa26m/1j\nVypzxlUGqHVaBQbnd//gcgryOqc6KHw/Ccn4e845z5nTcr4859UiIgIiIiITUmZ7BYiI6O7FECEi\nItMYIkREZBpDhIiITGOIEBGRaQwRIiIybUYhcuPGDWiahg0bNgAAnn76aSxatAiapkHTNHz++ecA\nABFBfX09VFXF0qVLcerUKWMZ+/btg6qqUFUV+/fvN+odHR3QNA2qqqKhocGo9/X1oaysDPn5+Sgv\nL8eVK1eMtsn6ICKi5JpRiOzevRu5ubmwWCwAAIvFgrfeegunTp3CqVOnUFBQAAA4cuQIurq68Oc/\n/xm//e1v8cwzzwAA/vGPf2Dbtm1ob29He3s7fvGLX6C7uxsA8Mwzz2DPnj3485//jAsXLuDo0aMA\ngNdeew3r1q3DmTNnUFFRgddeew0AcPjw4Qn7ICKi5Js2RHRdh8/nw3PPPYfR9yVOdI+iz+dDbW0t\nAEDTNAwNDUHXdbS1taGiogLp6elIT0/H2rVr0draiq6uLsTjcWiaBgCoqamB1+sdt6zRda/XO2Ef\nRESUfNOGyNatW7Fz506kpIyd9JVXXoHT6URdXR0GBgYADAeOw+EwplEUBbquIxwOQ1GUCeujp7fb\n7UYgRCIRZGRkAACsVqsxcrl5npFlERFR8k0ZIh9++CFsNhs0TRsz8tixYwf++te/4vPPP8f169ex\nbds2oy0ZT1G5uY+Rw2xERJRcaVM1Hjt2DB988AF8Ph/6+/tx9epV/OxnPzNOjN9777149tln0dTU\nBGB4VBAKhbBixQoA345MFEVBe3u7sdxQKISVK1ca048YPZLJzMxET08PrFYrIpEIbDbbpH2MHuWM\n+Od//mf8/e9/N71hiIjmo5ycHPztb3+b+QwyQ4FAQNavXy8iIpcvXxYRkXg8Lv/+7/8uW7duFRGR\nQ4cOSVVVlYiIdHR0SH5+voiIhMNhycnJkatXr8rVq1dl0aJFcunSJRERcblcEgwGRUSksrJSDh8+\nLCIidXV10tzcLCIiu3btkpdeemnKPm6WwFeb81577bXZXoU7BrfFt7gtvsVt8a1E951TjkRuChvj\nsFF1dTX6+vpw/fp1aJqG3/zmNwCATZs24ZNPPoGqqrjvvvuwd+9eAMAPf/hDvPLKK8bo4dVXX0VW\nVhYAYO/evdiyZQsGBwexevVq/OQnPwEANDU1wePxYM+ePXjwwQdx8ODBKfsgIqLkm3GIlJaWorS0\nFADw8ccfTzrdO++8M2H9mWeemfBy3H/5l3+Z8F6PBx54AG1tbQn1QUREycU71ueBkfAnbovRuC2+\nxW1hnuX/j4HNORaLJSlXihERzSWJ7js5EiEiItMYIkREZBpDhIiITGOIEBGRaQwRIiIyjSFCRESm\nMUSIiMg0hggREZnGECEiItMYIkREZNq8DJEFCx6AxWIZ97NgwQOzvWpERHeVefnsrOFH2k/Uxudt\nEdH8xmdnERFR0jBEiIjItBmFyI0bN6BpGjZs2AAA6OzsRElJCVwuF6qrqxGLxQAAAwMD8Hg8cLlc\nePTRR3HhwgVjGW+88QZyc3PhcrnQ2tpq1P1+P1wuF3Jzc7Fjxw6jbqYPIiJKrhmFyO7du5Gbm2u8\nHre+vh4vv/wyzp49iwcffNB40+A777yDhx56CGfPnsV//Md/oL6+HgDQ0dGBI0eO4OzZs/D7/Xj+\n+ecRi8UwMDCAF154AX6/H2fOnMGhQ4eMtxwm2gcRESXftCGi6zp8Ph+ee+45iAhu3LiB48ePo6qq\nCgBQU1MDr9cLAPD5fKitrQUAbNy4EceOHUM8HofX60V1dTVSU1Nht9uhqira29vR3t4OVVVht9uR\nlpYGj8cDr9eLoaGhhPrgyXAiotkxbYhs3boVO3fuRErK8KTd3d2wWq1Gu91uh67rAIYDx+FwDC84\nJQUZGRno7u5GOByGoijGPIqiQNd1hMNhY/rR9UgkknAfRESUfGlTNX744Yew2WzQNA2BQAAA7qq/\n+hsbG43PpaWlfI8yEdFNAoGAsX83Y8oQOXbsGD744AP4fD709/fj6tWrePnll9HT02NMo+u6McpQ\nFAVdXV2w2WyIx+Po7e1FZmYmFEVBKBQaM4/D4UA8Hh9TD4VCcDgcsNlsCfcxkdEhQkRE4938B3ZT\nU1NC8095OGv79u0IhULo7OzEH/7wBzzxxBP4/e9/j+LiYrz33nsAgJaWFrjdbgCA2+1GS0sLAOD9\n999HSUkJUlNT4Xa78e6772JoaAi6ruPcuXNYvnw5li1bhnPnziEcDiMWi+HgwYOoqKhAampqQn2M\nHGojIqIkkxkKBAKyYcMGERE5f/68FBcXS15enng8HhkcHBQRkf7+ftm8ebPk5eVJSUmJdHZ2GvO/\n/vrr4nQ6RVVV8fv9Rt3n84mqquJ0OmX79u1G3Uwfo0311QAIIBP8zHhzEBHNSYnuB/nYk7Etd9U5\nHyKiW42PPSEioqRhiBARkWkMESIiMo0hQkREpjFEiIjINIYIERGZxhAhIiLTGCJERGQaQ4SIiExj\niBARkWkMESIiMo0hQkREpjFEiIjINIYIERGZxhAhIiLTGCJERGTalCHS39+PZcuWQdM0PPLII9i6\ndSsA4Omnn8aiRYugaRo0TcPnn38OABAR1NfXQ1VVLF26FKdOnTKWtW/fPqiqClVVsX//fqPe0dEB\nTdOgqioaGhqMel9fH8rKypCfn4/y8nJcuXLFaJusDyIiSrLpXn147do1ERGJxWKyYsUK+fjjj+Xp\np5+Ww4cPj5v20KFDUllZKSIiwWBQCgoKRETk4sWLkpOTI9FoVKLRqOTk5Mjly5dFRMTlckkwGBQR\nkcrKSjly5IiIiNTV1Ulzc7OIiDQ3N0t9ff2Ufdxsqq8Gvh6XiGhCie4Hpz2c9f3vfx8AMDg4iBs3\nbsBms42Ez7hpfT4famtrAQCapmFoaAi6rqOtrQ0VFRVIT09Heno61q5di9bWVnR1dSEej0PTNABA\nTU0NvF7vuGWNrnu93gn7ICKi5Js2ROLxOAoLC5GVlYVVq1ZBVVUAwCuvvAKn04m6ujoMDAwAAHRd\nh8PhMOZVFAW6riMcDkNRlAnro6e32+1GIEQiEWRkZAAArFYruru7AWDcPCPLIiKi5EubboKUlBSc\nPn0aX331FcrLyxEIBLBjxw7YbDYMDg7ihRdewLZt2/Bf//VfACYeodxqN/dhsVgmnK6xsdH4XFpa\nitLS0tu4VkREd59AIIBAIGB6/mlDZMT999+PdevW4fjx48bO+N5778Wzzz6LpqYmAMOjglAohBUr\nVgD4dmSiKAra29uNZYVCIaxcudKYfsTokUxmZiZ6enpgtVoRiUSMw2gT9TF6lDPa6BAhIqLxbv4D\ne2R/PlNTHs7q7e1FNBoFAFy/fh1tbW1wuVzGoSURwZEjR4xDXG63GwcOHAAABINBpKamwm63Y/Xq\n1fD7/YhGo4hGo/D7/VizZg0cDgdSUlKMK6wOHDiAiooKY1ktLS0AgJaWFrjd7in7ICKi5JtyJHLx\n4kX87Gc/g4igv78fTz31FNatW4cnnngCfX19uH79OjRNw29+8xsAwKZNm/DJJ59AVVXcd9992Lt3\nLwDghz/8IV555RVj9PDqq68iKysLALB3715s2bIFg4ODWL16NX7yk58AGE5Dj8eDPXv24MEHH8TB\ngwen7IOIiJLPIsk4iTELLBbLpOdnhs+hTNQ2+TxERPPBVPvOifCOdSIiMo0hQkREpjFEiIjINIYI\nERGZxhAhIiLTGCJERGQaQ4SIiExjiBARkWkMESIiMo0hQkREpjFEiIjINIYIERGZxhAhIiLTGCJE\nRGQaQ4SIiExjiBARkWnThkh/fz+WLVsGTdPwyCOPYOvWrQCAzs5OlJSUwOVyobq6GrFYDAAwMDAA\nj8cDl8uFRx99FBcuXDCW9cYbbyA3Nxculwutra1G3e/3w+VyITc3Fzt27DDqZvogIqIkkhm4du2a\niIjEYjFZsWKFfPzxx7J+/Xo5evSoiIg0NDTIrl27RETkrbfekoaGBhEROXr0qGzcuFFERE6ePClF\nRUUyNDQkuq5Ldna2DA4OSn9/v2RnZ4uu6xKLxaSoqEiCwaCISMJ9jDbVVwMggEzwM6PNQUQ0ZyW6\nH5zR4azvf//7AIDBwUHcuHEDNpsNx48fR1VVFQCgpqYGXq8XAODz+VBbWwsA2LhxI44dO4Z4PA6v\n14vq6mqkpqbCbrdDVVW0t7ejvb0dqqrCbrcjLS0NHo8HXq8XQ0NDCfUhfK0tEVHSzShE4vE4CgsL\nkZWVhVWrVmHhwoWwWq1Gu91uh67rAABd1+FwOIYXnpKCjIwMdHd3IxwOQ1EUYx5FUaDrOsLhsDH9\n6HokEkm4DyIiSq60mUyUkpKC06dP46uvvkJ5eTkKCwtv93rdEo2Njcbn0tJSlJaWztq6EBHdiQKB\nAAKBgOn5ZxQiI+6//36sW7cO58+fR09Pj1HXdd0YZSiKgq6uLthsNsTjcfT29iIzMxOKoiAUCo2Z\nx+FwIB6Pj6mHQiE4HA7YbLaE+7jZ6BAhIqLxbv4Du6mpKaH5pz2c1dvbi2g0CgC4fv062traUFhY\niOLiYrz33nsAgJaWFrjdbgCA2+1GS0sLAOD9999HSUkJUlNT4Xa78e6772JoaAi6ruPcuXNYvnw5\nli1bhnPnziEcDiMWi+HgwYOoqKhAampqQn2kpPBqZSKipJvuzPuZM2eksLBQCgoKZPHixdLU1CQi\nIufPn5fi4mLJy8sTj8cjg4ODIiLS398vmzdvlry8PCkpKZHOzk5jWa+//ro4nU5RVVX8fr9R9/l8\noqqqOJ1O2b59u1E308dMrjAAr84iIppQovtBy//PNOdYLJZJr9iyWCwAJmqbfB4iovlgqn3nRHgM\niIiITGOIEBGRaQwRIiIyjSFCRESmMUSIiMg0hggREZnGECEiItMYIkREZBpDhIiITGOIEBGRaQwR\nIiIyjSEyRhosFsuEPwsWPDDbK0dEdMfhAxjHtkxSH26bo5uKiMjABzASEVHSMESIiMi0KUMkFArh\n8ccfh8vlwuLFi/Hmm28CGH7trKIo0DQNmqbho48+MuZ54403kJubC5fLhdbWVqPu9/vhcrmQm5uL\nHTt2GPXOzk6UlJTA5XKhuroasVgMADAwMACPxwOXy4VHH30UFy5cmLYPIiJKsqneWHXp0iU5e/as\niIhEo1F5+OGH5fTp09LY2Chvv/32uOlPnjwpRUVFMjQ0JLquS3Z2tgwODkp/f79kZ2eLrusSi8Wk\nqKhIgsGgiIisX79ejh49KiIiDQ0NsmvXLhEReeutt6ShoUFERI4ePSobN26ctI+BgYGE3s6FKd5s\nOHGdbz0kovkh0X3dlCORrKws5OXlAQDS09ORn5+PcDg8Ej7jpvd6vaiurkZqairsdjtUVUV7ezva\n29uhqirsdjvS0tLg8Xjg9XoxNDSE48ePo6qqCgBQU1MDr9cLAPD5fKitrQUAbNy4EceOHUM8Hp+w\njz/96U+3JFCJiCgxMz4n8sUXX+DEiRN47LHHAAC/+tWv4HQ6UVNTg76+PgBAOByGoijGPIqiQNd1\nhMNhOByOcfVIJAKr1WrU7XY7dF0HAOi6bsyTkpKCjIwMdHd3T9oHEREl34xC5Ouvv8bmzZuxe/du\n/OAHP8CLL76Iv//97/jLX/6CnJwc1NfX3+71JCKiO1DadBPEYjFs2rQJTz31lHHYafTo4fnnn8eq\nVasADI8KQqGQ0TYymojH42PqoVAIDocDNpsNPT09Y6YfGWUoioKuri7YbDbE43H09vYiMzNz0j4m\n0tjYaHwuLS1FaWnpdF+XiGheCQQCCAQC5hcw1QmTeDwutbW18vOf/3xM/fLly8bnX/7yl/Kv//qv\nIvLtSe9YLCahUEh+9KMfyeDgoFy/fl1+9KMfia7rMjg4KEVFRdLR0SEiY0+s19fXGyfsR59YP3Lk\niGzYsGHKPhI5OQSeWCcimlCi+7opp/7ss8/EYrFIQUGBFBYWSmFhofh8PqmpqZH8/HxZsmSJlJeX\ni67rxjyvv/66OJ1OUVVV/H6/Uff5fKKqqjidTtm+fbtRP3/+vBQXF0teXp54PB4jEPr7+2Xz5s2S\nl5cnJSUl0tnZOW0fM90QDBEiookluq/jY0/GtkxSH26bo5uKiMjAx54QEVHSMESIiMg0hggREZnG\nECEiItMYIkREZBpDhIiITGOIEBGRaQwRIiIyjSFCRESmMUSIiMg0hggREZnGECEiItMYIkREZBpD\nhIiITGOIEBGRaQwRIiIybdoQCYVCePzxx+FyubB48WK8+eabAIC+vj6UlZUhPz8f5eXluHLlijFP\nfX09VFXF0qVLcerUKaO+b98+qKoKVVWxf/9+o97R0QFN06CqKhoaGoy6mT6IiCiJpnv14aVLl+Ts\n2bMiIhKNRuXhhx+W06dPS11dnTQ3N4uISHNzs9TX14uIyKFDh6SyslJERILBoBQUFIiIyMWLFyUn\nJ0ei0ahEo1HJyckx3tXucrkkGAyKiEhlZaUcOXJERCThPmb6ikfw9bhERBNKdF837UgkKysLeXl5\nAID09HTk5+cjHA7D5/OhtrYWAFBTUwOv1wsA8Hq9Rl3TNAwNDUHXdbS1taGiogLp6elIT0/H2rVr\n0draiq6uLsTjcWiaNm5ZifZBRETJldA5kS+++AInTpzAj3/8Y0QiEWRkZAAArFYruru7AQDhcBgO\nh8OYR1EU6LqOcDgMRVEmrI+e3m63G4GQaB9ERJRcMw6Rr7/+Gk8++SR2796NBQsWTDmtJPCSd7Nu\n7sNisdz2PomIaKy0mUwUi8WwadMm/PSnP0VVVRUAIDMzEz09PbBarYhEIrDZbACGRwWhUAgrVqwA\nAOi6DofDAUVR0N7ebiwzFAph5cqVxvQjRqZPtI/Ro5wRjY2NxufS0lKUlpbOdLsQEc0LgUAAgUDA\n/AKmO2kSj8eltrZWfv7zn4+pjz7pvWvXLnnppZdEZPikd1VVlYiIdHR0SH5+voiIhMNhycnJkatX\nr8rVq1dl0aJFcunSJREZf2L98OHDpvoYbaqvBp5YJyKaUKL7ummn/uyzz8RisUhBQYEUFhZKYWGh\nfPTRR9Lb2ytr1qwRl8slZWVl8uWXXxrzvPjii5KbmyuapklHR4dR37NnjzidTnE6nfK73/3OqJ88\neVIKCwslNzfXCAoRMdXHTDYEQ4SIaGKJ7uss/z/TnGOxWCY9NzN8/mSitsnqw21zdFMRERmm2ndO\nhHesExGRaQwRIiIyjSFCRESmMUSIiMg0hggREZnGECEiItMYIkREZBpDhIiITGOIEBGRaQwRIiIy\njSFCRESmMUSIiMg0hggREZnGECEiItMYIjOWBovFMuHPggUPzPbKERHNCr5PZGzLJPXp2+boZiSi\neeaWv09ky5YtyMrKgsvlMmqNjY1QFAWapkHTNHz00UdG2xtvvIHc3Fy4XC60trYadb/fD5fLhdzc\nXOzYscOod3Z2oqSkBC6XC9XV1YjFYgCAgYEBeDweuFwuPProo7hw4cK0fRARUZJN9+rDTz/9VILB\noOTl5Rm1xsZGefvtt8dNe/LkSSkqKpKhoSHRdV2ys7NlcHBQ+vv7JTs7W3Rdl1gsJkVFRcY71dev\nXy9Hjx4VEZGGhgbZtWuXiIi89dZb0tDQICIiR48elY0bN07ax8DAwLh1meqrweTrcfnqXCKa6xLd\nn007EnnsscewcOHCicJnXM3r9aK6uhqpqamw2+1QVRXt7e1ob2+Hqqqw2+1IS0uDx+OB1+vF0NAQ\njh8/jqqqKgBATU0NvF4vAMDn86G2thYAsHHjRhw7dgzxeHzCPv70pz+ZjFAiIvouTJ9Y/9WvfgWn\n04mamhr09fUBAMLhMBRFMaZRFAW6riMcDsPhcIyrRyIRWK1Wo26326HrOgBA13VjnpSUFGRkZKC7\nu3vSPoiIKPnSzMz04osv4tVXXwUwfH6kvr4eLS0tt3TFboXGxkbjc2lpKUpLS2dtXYiI7kSBQACB\nQMD0/KZCZPTo4fnnn8eqVasADI8KQqGQ0TYymojH42PqoVAIDocDNpsNPT09Y6YfGWUoioKuri7Y\nbDbE43H09vYiMzNz0j4mMjpEiIhovJv/wG5qakpoflOHs7q7u43Phw8fhqqqAAC32413330XQ0ND\n0HUd586dw/Lly7Fs2TKcO3cO4XAYsVgMBw8eREVFBVJTU1FcXIz33nsPANDS0gK3220sa2R08/77\n76OkpASpqamT9kFERLNgujPv1dXV8tBDD8k999wjiqLIb3/7W6mpqZH8/HxZsmSJlJeXi67rxvSv\nv/66OJ1OUVVV/H6/Uff5fKKqqjidTtm+fbtRP3/+vBQXF0teXp54PB4ZHBwUEZH+/n7ZvHmz5OXl\nSUlJiXR2dk7bx0yvMACvziIimlCi+zPebDi2ZZL69G1zdDMS0Txzy282JCIimgxDhIiITGOIEBGR\naQwRIiIyjSFCRESmMUSIiMg0hggREZnGECEiItMYIkREZBpDhIiITGOI3BJpsFgs434WLHhgtleM\niOi24rOzxrZMUjfbxmdqEdHdhc/OIiKipGGIEBGRaQwRIiIyjSFCRESmTRsiW7ZsQVZWFlwul1Hr\n6+tDWVkZ8vPzUV5ejitXrhht9fX1UFUVS5cuxalTp4z6vn37oKoqVFXF/v37jXpHRwc0TYOqqmho\naPhOfRARUZJN9+rDTz/9VILBoOTl5Rm1uro6aW5uFhGR5uZmqa+vFxGRQ4cOSWVlpYiIBINBKSgo\nEBGRixcvSk5OjkSjUYlGo5KTkyOXL18WERGXyyXBYFBERCorK+XIkSOm+rjZVF8Nt+H1uHxtLhHN\nBYnut6YdiTz22GNYuHDhmJrP50NtbS0AoKamBl6vFwDg9XqNuqZpGBoagq7raGtrQ0VFBdLT05Ge\nno61a9eitbUVXV1diMfj0DRt3LIS7YOIiJLP1DmRSCSCjIwMAIDVakV3dzcAIBwOw+FwGNMpigJd\n1xEOh6EoyoT10dPb7XYjEBLtg4iIki/tVi9QknBz3c19DN88OF5jY6PxubS0FKWlpbdxrYiI7j6B\nQACBQMD0/KZCJDMzEz09PbBarYhEIrDZbACGRwWhUAgrVqwAAOi6DofDAUVR0N7ebswfCoWwcuVK\nY/oRI9Mn2sfoUc5oo0OEiIjGu/kP7KampoTmN3U4y+12o6WlBQDQ0tICt9tt1A8cOAAACAaDSE1N\nhd1ux+rVq+H3+xGNRhGNRuH3+7FmzRo4HA6kpKQYV1gdOHAAFRUVpvogIqJZMN2Z9+rqannooYfk\nnnvuEUVRZM+ePdLb2ytr1qwRl8slZWVl8uWXXxrTv/jii5KbmyuapklHR4dR37NnjzidTnE6nfK7\n3/3OqJ88eVIKCwslNzdXXnrpJaNupo/Rpvpq4NVZREQTSnS/xQcwjm2ZpG62jQ9gJKK7Cx/ASERE\nScMQISIi0xgiRERkGkPktpr4jYd86yERzRU8sT62ZZK62bap55mjm56I7mI8sU5EREnDECEiItMY\nIkREZBpDhIiITGOIEBGRaQyRWcPLf4no7nfL3ydCMzWEyS7/jUYnfj8KEdGdhiMRIiIyjSFCRESm\nMUSIiMg0hggREZn2nUIkOzsb+fn50DQNy5cvBwD09fWhrKwM+fn5KC8vx5UrV4zp6+vroaoqli5d\narwSFwD27dsHVVWhqir2799v1Ds6OqBpGlRVRUNDg1Gfqg8iIkqi7/IaxezsbOnt7R1Tq6urk+bm\nZhERaW5ulvr6ehEROXTokFRWVoqISDAYlIKCAhERuXjxouTk5Eg0GpVoNCo5OTly+fJlERFxuVwS\nDAZFRKSyslKOHDkyZR+jTfXVkMTX45pdHhHRbEh0//OdD2fJTU979Pl8qK2tBQDU1NTA6/UCALxe\nr1HXNA1DQ0PQdR1tbW2oqKhAeno60tPTsXbtWrS2tqKrqwvxeByapo1b1mR9EBFRcn2nELFYLMZh\npXfeeQcAEIlEkJGRAQCwWq3o7u4GAITDYTgcDmNeRVGg6zrC4TAURZmwPnp6u90OXden7GPu4I2I\nRHR3+E43Gx4/fhw2mw2RSARr167FkiVLppz+5lHL7dbY2Gh8Li0tRWlpaVL7N483IhJRcgQCAQQC\nAdPzf6cQsdlsAIDMzEw8+eSTOHHiBDIzM9HT0wOr1YpIJGJMoygKQqEQVqxYAQDQdR0OhwOKoqC9\nvd1YZigUwsqVK43pR4xMP9LfRH3cbHSIEBHReDf/gd3U1JTQ/KYPZ127dg3Xrl0DAHzzzTfw+/1Q\nVRVutxstLS0AgJaWFrjdbgCA2+3GgQMHAADBYBCpqamw2+1YvXo1/H4/otEootEo/H4/1qxZA4fD\ngZSUFOMqrgMHDqCiosJY1kR9EBFRcpl+PW5nZyeqqqpgsVhw7do1VFdX4xe/+AX6+vrg8Xhw+fJl\nPPjggzh48CD+6Z/+CQBQV1eHTz75BPfddx/+53/+B0uXLgUA7N27Fzt37gQAvPzyy/i3f/s3AMOX\n+D733HMYHBzE6tWr8ctf/hIApuzD+GJ3wetxzbXdg+HDXWP94AcLcfVq3yTzEBHNTKKvx+U71se2\nTFI323Y7QmTyvubof0oiSiK+Y52IiJKGIUJERKYxROYM3ltCRMnHl1LNGby3hIiSjyMRIiIyjSEy\nL/BQFxHdHjycNS/wUBcR3R4cicx7E49SOEIhoplgiMx7I6OUsT/RaJSHwIhoWjycRZPgITAimh5H\nImQCT9QT0TCGCJkw8SGwqQ6DMVyI5iaGCN1iPMdCNJ8wRChJEh+9MGCI7nwMEboDmAsYi+VeBg/R\nLLtrQ8SStX0LAAAFwklEQVTv98PlciE3Nxc7duyY7dWh22bygAFiE9bNBA/Dh8icuzJEBgYG8MIL\nL8Dv9+PMmTM4dOiQ8RpdIjPBM3X4TB48ZtrulLAKBAKzvQp3DG4L8+7KEGlvb4eqqrDb7UhLS4PH\n44HX653t1aK73mThM3nwmGkzO1K61YHFHee3uC3MuytDRNd1OBwO49+KokDX9VlcI6JEmBsp3erA\namraZiKwbm3IJXN5d8oIMFkWLHggKdvirrxj3WKZ2R3TGzZsGFcrKiq61atDdAeb/MkDgMVEm5l5\n7ozlRaP3TLHvSEFTU9MkbfdgOKRnWjfbdquXByTjqRN3ZYgoioJQKGT8OxQKjRmZAEBOTg4+/PDD\ncfN+W5tsI061cc203erlJbOvO315yezrTl9eMvu6m5c3mfgUbZPtoCerm2271csDptoWkwVqTk7O\nFMubYDkiMlnU37H6+/uxZMkS/O///i9sNhtWrlyJ//7v/8bSpUtne9WIiOaVu3Ik8r3vfQ+//vWv\nUV5ejng8jtraWgYIEdEsuCtHIkREdGe4K6/Oms58vhFxy5YtyMrKgsvlMmp9fX0oKytDfn4+ysvL\nceXKlVlcw+QIhUJ4/PHH4XK5sHjxYrz55psA5ue26O/vx7Jly6BpGh555BFs3boVANDZ2YmSkhK4\nXC5UV1cjFpvq2PrccuPGDWiaZlx8M1+3RXZ2NvLz86FpGpYvXw7AxO+IzDH9/f2SnZ0tuq5LLBaT\noqIiCQaDs71aSfPpp59KMBiUvLw8o1ZXVyfNzc0iItLc3Cz19fWztXpJc+nSJTl79qyIiESjUXn4\n4Yfl9OnT83JbiIhcu3ZNRERisZisWLFCPv74Y1m/fr0cPXpUREQaGhpk165ds7mKSfX222/LU089\nJRs2bBARmbfbIjs7W3p7e8fUEv0dmXMh8sc//lHWrVtn/Hvnzp2ybdu2WVyj5Ovs7BwTIosWLZKe\nnh4REYlEIpKTkzNbqzZrNm3aJF6vd95vi2+++UaKiork3LlzYrVajfqJEydk9erVs7hmyRMKhWT1\n6tVGkA4NDc3bbZGdnW38PoxI9Hdkzh3O4o2I40UiEWRkZAAArFYruru7Z3mNkuuLL77AiRMn8OMf\n/3jebot4PI7CwkJkZWVh1apVWLhwIaxWq9Fut9vnze/J1q1bsXPnTqSkDO/+uru75+22sFgsxqGr\nd955B0Di+4u78uqsqcz0RkSaH77++ms8+eST2L17NxYsWDDbqzNrUlJScPr0aXz11VcoLy9HYWHh\nbK/SrPjwww9hs9mgaZrxqBOZx9cWHT9+HDabDZFIBGvXrsWSJUsSXsacC5GZ3Ig432RmZqKnpwdW\nqxWRSAQ2m222VykpYrEYNm3ahJ/+9KeoqqoCMH+3xYj7778f69atw/nz59HT02PUdV2HoiizuGbJ\ncezYMXzwwQfw+Xzo7+/H1atX8fLLL8/LbQHA+P8/MzMTTz75JE6cOJHw78icO5y1bNkynDt3DuFw\nGLFYDAcPHkRFRcVsr9ascrvdaGlpAQC0tLTA7XbP8hrdfiKCZ599Frm5ucbVSMD83Ba9vb2IRqMA\ngOvXr6OtrQ2FhYUoLi7Ge++9B2D+bIvt27cjFAqhs7MTf/jDH/DEE0/g97///bzcFteuXcO1a9cA\nAN988w38fj9UVU38d+Q2na+ZVT6fT1RVFafTKdu3b5/t1Umq6upqeeihh+See+4RRVFkz5490tvb\nK2vWrBGXyyVlZWXy5ZdfzvZq3nafffaZWCwWKSgokMLCQiksLJSPPvpoXm6LM2fOSGFhoRQUFMji\nxYulqalJRETOnz8vxcXFkpeXJx6PRwYHB2d5TZMrEAgYV2fNx21x/vx5yc/Pl4KCAnn44YflP//z\nP0VEEv4d4c2GRERk2pw7nEVERMnDECEiItMYIkREZBpDhIiITGOIEBGRaQwRIiIyjSFCRESmMUSI\niMi0/wNUZs8inVyTMQAAAABJRU5ErkJggg==\n",
"text": [
"<matplotlib.figure.Figure at 0x392cd50>"
]
}
],
"prompt_number": 27
},
{
"cell_type": "heading",
"level": 4,
"metadata": {},
"source": [
"Why are there so many pages that are linked only once?"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"linked_only_once = number_of_inlinks.filter(lambda (title, count): count == 1).map(lambda x: x[0])"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 29
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"wiki_table(linked_only_once.take(20))"
],
"language": "python",
"metadata": {},
"outputs": [
{
"html": [
"<table><tr><td><a href=\"http://en.wikipedia.org/wiki/pl:Boniewo-Kolonia\" target=\"_BLANK\">pl:Boniewo-Kolonia</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Jean Laurent Heusse\" target=\"_BLANK\">Jean Laurent Heusse</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/pt:Suchowola\" target=\"_BLANK\">pt:Suchowola</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/thallassocrathic\" target=\"_BLANK\">thallassocrathic</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Pharnaspes\" target=\"_BLANK\">Pharnaspes</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Gary Tallon\" target=\"_BLANK\">Gary Tallon</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Nampe\" target=\"_BLANK\">Nampe</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Games-X\" target=\"_BLANK\">Games-X</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/fi:Herbert Reinecker\" target=\"_BLANK\">fi:Herbert Reinecker</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Arunbhai Gujarati\" target=\"_BLANK\">Arunbhai Gujarati</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Hypalastoroides paraguayensis\" target=\"_BLANK\">Hypalastoroides paraguayensis</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Albert Mando\" target=\"_BLANK\">Albert Mando</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/de:Hans Herrmann\" target=\"_BLANK\">de:Hans Herrmann</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/\u041f\u0443\u0441\u0442\u044c \u043c\u0438p\u043e\u043c \u043fpa\u0432\u0438\u0442 \u043b\u044e\u0431\u043e\u0432\u044c\" target=\"_BLANK\">\u041f\u0443\u0441\u0442\u044c \u043c\u0438p\u043e\u043c \u043fpa\u0432\u0438\u0442 \u043b\u044e\u0431\u043e\u0432\u044c</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Funadhoo (Baa Atoll)\" target=\"_BLANK\">Funadhoo (Baa Atoll)</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/eu:Petxora\" target=\"_BLANK\">eu:Petxora</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/ms:.re\" target=\"_BLANK\">ms:.re</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/ja:\u5168\u56fd\u5730\u57df\u30ea\u30fc\u30b0\u6c7a\u52dd\u5927\u4f1a (\u30b5\u30c3\u30ab\u30fc)\" target=\"_BLANK\">ja:\u5168\u56fd\u5730\u57df\u30ea\u30fc\u30b0\u6c7a\u52dd\u5927\u4f1a (\u30b5\u30c3\u30ab\u30fc)</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/oc:696\" target=\"_BLANK\">oc:696</a></td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/national Red Cross societies\" target=\"_BLANK\">national Red Cross societies</a></td></tr></table>"
],
"metadata": {},
"output_type": "pyout",
"prompt_number": 30,
"text": [
"<IPython.core.display.HTML at 0x3b95410>"
]
}
],
"prompt_number": 30
},
{
"cell_type": "markdown",
"metadata": {},
"source": [
"Many of these pages don't even exist! Wikilinks may point to pages that don't exist, so let's filter those pages out and see if it changes the histogram:"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"page_titles = wiki.map(lambda x: x.split(\"\\t\")[0]).cache()"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 31
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"number_of_inlinks_for_existing_pages = number_of_inlinks. \\\n",
" leftOuterJoin(page_titles.map(lambda x: (x, True))). \\\n",
" filter(lambda (title, (count, exists)): exists is not None). \\\n",
" map(lambda (title, (count, exists)): (title, count))"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 32
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"%time inlink_frequencies_for_existing_pages = number_of_inlinks_for_existing_pages.map(lambda x: x[1]).countByValue()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"CPU times: user 212 ms, sys: 36 ms, total: 248 ms\n",
"Wall time: 28.4 s\n"
]
}
],
"prompt_number": 33
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"pylab.hist(inlink_frequencies_for_existing_pages.keys(), bins=range(50), weights=inlink_frequencies_for_existing_pages.values());"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "display_data",
"png": "iVBORw0KGgoAAAANSUhEUgAAAYUAAAEACAYAAABcXmojAAAABHNCSVQICAgIfAhkiAAAAAlwSFlz\nAAALEgAACxIB0t1+/AAAIABJREFUeJzt3XtsVGX+BvBnWiprUvCHlKk6Z2JDV5EepvSUcl0lg0BK\nh0uruGkXW1fRxBhrK9kYsyG67bqRIJcGgiGaFRYsRgmCGqc2bSKj6xIa7ICAm/1jBWXOsNCbyCCU\nTpnv74/KawudXqa3KfN8kkmG95zzzntOmfPMe95zsYiIgIiICEDcSDeAiIiiB0OBiIgUhgIRESkM\nBSIiUhgKRESkMBSIiEjpUyhcu3YNhmFg+fLlAIAnn3wSkydPhmEYMAwD33zzDQBARFBSUgJd15GZ\nmYmjR4+qOnbt2gVd16HrOnbv3q3K6+vrYRgGdF1HaWnpYK4bERH1U59CYcuWLUhLS4PFYgEAWCwW\nbNy4EUePHsXRo0cxffp0AMD+/ftx5swZfPvtt3jnnXfw1FNPAQD+97//4bXXXkNdXR3q6urw17/+\nFQ0NDQCAp556Cjt27MC3336LH374AQcOHBiK9SQioj7oNRRM00RVVRWeeeYZdL7Orbtr3qqqqlBU\nVAQAMAwD7e3tME0TtbW1yMnJQWJiIhITE7FkyRLU1NTgzJkzCIVCMAwDAFBYWAi32z1Y60ZERP3U\nayisWbMGGzZsQFxc11nXrl2LqVOnori4GFevXgXQESB2u13No2kaTNOE3++Hpmndlnee32azwTTN\nAa8UERFFpsdQ+PTTT2G1WmEYRpeewfr16/Gf//wH33zzDa5cuYLXXntNTeNdM4iIRq8xPU08dOgQ\nPvnkE1RVVaG1tRUXL17EE088oQaKb7vtNjz99NMoLy8H0NED8Pl8mD17NoBfew6apqGurk7V6/P5\nMG/ePDX/daZpdulRdPbb3/4W33333cDWlogohqSmpuK///1v/xaSPvJ4PLJs2TIRETl//ryIiIRC\nIfnTn/4ka9asERGRffv2SV5enoiI1NfXS3p6uoiI+P1+SU1NlYsXL8rFixdl8uTJcu7cORERcTgc\n4vV6RUQkNzdXPvzww24/vx9NvaX95S9/GekmRA1ui19xW/yK2+JXkew3e+wp3BAe6uyjgoICtLS0\n4MqVKzAMA2+//TYAYOXKlTh48CB0XcfYsWOxc+dOAMA999yDtWvXqh7Eq6++iuTkZADAzp07sXr1\narS1tWHhwoV49NFH+5dqREQ0aPocCk6nE06nEwDw+eefh51v27Zt3ZY/9dRT6hTVzmbMmNHlegYi\nIho5vKJ5lLkezMRt0Rm3xa+4LQbG8stxp6hnsVh4ZhMRUT9Est9kT4GIiBSGAhERKQwFIiJSGApE\nRKSM+lAYP/5OWCyWbl/jx9850s0jIhpVRv3ZRx0X1IVbBZ6xRESxi2cfERHRgNzioTCGh5aIiPrh\nlj98xENLRBSrePiIiIgGhKFAREQKQ4GIiBSGAhERKQwFIiJS+hQK165dg2EYWL58OQDg9OnTmDt3\nLhwOBwoKChAMBgEAV69eRX5+PhwOB373u9/hhx9+UHWsW7cOaWlpcDgcqKmpUeXV1dVwOBxIS0vD\n+vXrB3PdiIion/oUClu2bEFaWpp6HGdJSQlefvllnDhxAnfddZd62tq2bdtw991348SJE3jppZdQ\nUlICAKivr8f+/ftx4sQJVFdX49lnn0UwGMTVq1fx3HPPobq6GsePH8e+ffv4FDYiohHUayiYpomq\nqio888wzEBFcu3YNhw8fRl5eHgCgsLAQbrcbAFBVVYWioiIAwIoVK3Do0CGEQiG43W4UFBQgPj4e\nNpsNuq6jrq4OdXV10HUdNpsNY8aMQX5+vqqLiIiGX6+hsGbNGmzYsAFxcR2zNjQ0ICkpSU232Www\nTRNAR4DY7faOiuPiMHHiRDQ0NMDv90PTNLWMpmkwTRN+v1/N37mciIhGxpieJn766aewWq0wDAMe\njwcARvQq4LKyMvXe6XTyWaxERJ14PB61r45Uj6Fw6NAhfPLJJ6iqqkJraysuXryIl19+GU1NTWoe\n0zRVL0DTNJw5cwZWqxWhUAjNzc2YNGkSNE2Dz+frsozdbkcoFOpS7vP5uvQcbtQ5FIiIqKsbfyyX\nl5f3u44eDx+9/vrr8Pl8OH36NN5//308/PDDePfddzFnzhx89NFHAIDKykq4XC4AgMvlQmVlJQDg\n448/xty5cxEfHw+Xy4UPPvgA7e3tME0TJ0+exKxZszBz5kycPHkSfr8fwWAQe/fuRU5OTr9XgoiI\nBkePPYUbXT/7aOvWrVi1ahVeeeUV6LqOjRs3AgCKi4tRVFQEh8OBcePG4b333gMAzJgxA4888gjS\n09MRFxeHt956CwkJCUhISMD27duRnZ2NUCiEoqIiZGZmDvIqEhFRX/EuqUREtyjeJZWIiAaEoUBE\nRApDgYiIFIYCEREpDAUiIlIYCkREpDAUiIhIYSgQEZHCUCAiIoWhQERECkOBiIgUhgIRESkMBSIi\nUhgKRESkMBSIiEhhKBARkdJjKLS2tmLmzJkwDAP3338/1qxZAwB48sknMXnyZBiGAcMw8M033wAA\nRAQlJSXQdR2ZmZk4evSoqmvXrl3QdR26rmP37t2qvL6+HoZhQNd1lJaWDsU6EhFRX0kvLl++LCIi\nwWBQZs+eLZ9//rk8+eST8uGHH9407759+yQ3N1dERLxer0yfPl1ERM6ePSupqakSCAQkEAhIamqq\nnD9/XkREHA6HeL1eERHJzc2V/fv3d9uOcE0FIICEefU8jYjoVhbJfq7Xw0e33347AKCtrQ3Xrl2D\n1Wq9HiY3zVtVVYWioiIAgGEYaG9vh2maqK2tRU5ODhITE5GYmIglS5agpqYGZ86cQSgUgmEYAIDC\nwkK43e7ByDoiIopAr6EQCoWQkZGB5ORkLFiwALquAwDWrl2LqVOnori4GFevXgUAmKYJu92ultU0\nDaZpwu/3Q9O0bss7z2+z2WCa5qCtHBER9c+Y3maIi4vDsWPH8NNPPyE7Oxsejwfr16+H1WpFW1sb\nnnvuObz22mv429/+BqD7HsRgKSsrU++dTiecTueQfRYR0Wjj8Xjg8XgGVEevoXDdHXfcgaVLl+Lw\n4cNqZ3zbbbfh6aefRnl5OYCOHoDP58Ps2bMB/Npz0DQNdXV1qi6fz4d58+ap+a8zTbNLj+JGnUOB\niIi6uvHH8vV9c3/0ePioubkZgUAAAHDlyhXU1tbC4XCgoaEBQEevYP/+/eqQksvlwp49ewAAXq8X\n8fHxsNlsWLhwIaqrqxEIBBAIBFBdXY1FixbBbrcjLi5OnaW0Z88euFyufq8EERENjh57CmfPnsUT\nTzwBEUFraytWrVqFpUuX4uGHH0ZLSwuuXLkCwzDw9ttvAwBWrlyJgwcPQtd1jB07Fjt37gQA3HPP\nPVi7dq3qQbz66qtITk4GAOzcuROrV69GW1sbFi5ciEcffXQo15eIiHpgkaEcBBhEFoul2/EKi8UC\nINwq9DxtlKw6EVFEwu03e8IrmomISInhUBgDi8Vy02v8+DtHumFERCMmpg8fdT+Nh5WI6NbAw0dE\nRDQgDAUiIlIYCkREpDAUbtL9ADQHoYkoFnCguZ/LjJLNRUTEgWYiIhoYhgIRESkMBSIiUhgK/cJB\naCK6tXGgeVCW6Zg2SjYlEcUIDjQTEdGAMBSIiEhhKBARkdJrKLS2tmLmzJkwDAP3338/1qxZAwA4\nffo05s6dC4fDgYKCAgSDQQDA1atXkZ+fD4fDgd/97nf44YcfVF3r1q1DWloaHA4HampqVHl1dTUc\nDgfS0tKwfv36wV5HIiLqK+mDy5cvi4hIMBiU2bNny+effy7Lli2TAwcOiIhIaWmpbN68WURENm7c\nKKWlpSIicuDAAVmxYoWIiHz99deSlZUl7e3tYpqmpKSkSFtbm7S2tkpKSoqYpinBYFCysrLE6/Xe\n1IZwTQUggIR5RTIt8vqIiKJJJPulPh0+uv322wEAbW1tuHbtGqxWKw4fPoy8vDwAQGFhIdxuNwCg\nqqoKRUVFAIAVK1bg0KFDCIVCcLvdKCgoQHx8PGw2G3RdR11dHerq6qDrOmw2G8aMGYP8/HxVFxER\nDa8+hUIoFEJGRgaSk5OxYMECTJgwAUlJSWq6zWaDaZoAANM0YbfbOyqPi8PEiRPR0NAAv98PTdPU\nMpqmwTRN+P1+NX/nciIiGn5j+jJTXFwcjh07hp9++gnZ2dnIyMgY6nZ1q6ysTL13Op1wOp0j0g4i\nomjk8Xjg8XgGVEefQuG6O+64A0uXLsWpU6fQ1NSkyk3TVL0ATdNw5swZWK1WhEIhNDc3Y9KkSdA0\nDT6fr8sydrsdoVCoS7nP5+vSc+iscygQEVFXN/5YLi8v73cdvR4+am5uRiAQAABcuXIFtbW1yMjI\nwJw5c/DRRx8BACorK+FyuQAALpcLlZWVAICPP/4Yc+fORXx8PFwuFz744AO0t7fDNE2cPHkSs2bN\nwsyZM3Hy5En4/X4Eg0Hs3bsXOTk5/V4RIiIauF57CmfPnsUTTzwBEUFraytWrVqFZcuWQdd1rFq1\nCq+88gp0XcfGjRsBAMXFxSgqKoLD4cC4cePw3nvvAQBmzJiBRx55BOnp6YiLi8Nbb72FhIQEJCQk\nYPv27cjOzkYoFEJRUREyMzOHdq2JiKhbvPfRoCzTMW2UbEoiihG899GI6v4Oqrx7KhGNJv0aaKae\ntKO7XkQgYBn+phARRYg9BSIiUhgKRESkMBSIiEhhKBARkcJQICIihaFAREQKQ4GIiBSGAhERKQwF\nIiJSGApERKQwFIiISGEoEBGRwlAgIiKlx1Dw+XyYP38+HA4HpkyZgjfeeANAx2MxNU2DYRgwDAOf\nffaZWmbdunVIS0uDw+FATU2NKq+urobD4UBaWhrWr1+vyk+fPo25c+fC4XCgoKAAwWBwsNeRiIj6\nSnpw7tw5OXHihIiIBAIBue++++TYsWNSVlYmmzZtumn+r7/+WrKysqS9vV1M05SUlBRpa2uT1tZW\nSUlJEdM0JRgMSlZWlni9XhERWbZsmRw4cEBEREpLS2Xz5s3dtiVcUwEIIGFekUwb/PqIiEZCJPuf\nHnsKycnJmDZtGgAgMTER6enp8Pv918PkpvndbjcKCgoQHx8Pm80GXddRV1eHuro66LoOm82GMWPG\nID8/H263G+3t7Th8+DDy8vIAAIWFhXC73YMUd0RE1F99HlP4/vvvceTIETz00EMAgDfffBNTp05F\nYWEhWlpaAAB+vx+apqllNE2DaZrw+/2w2+03lTc2NiIpKUmV22w2mKY54JUiIqLI9CkULl26hN//\n/vfYsmULxo0bh+effx7fffcd/v3vfyM1NRUlJSVD3U4iIhoGvT6OMxgMYuXKlVi1apU6zNP51/2z\nzz6LBQsWAOjoAfh8PjXNNE3Y7XaEQqEu5T6fD3a7HVarFU1NTV3m79zTuFFZWZl673Q64XQ6e19D\nIqIY4fF44PF4BlZJTwMOoVBIioqK5MUXX+xSfv78efV+69at8sgjj4jIrwPNwWBQfD6f3HvvvdLW\n1iZXrlyRe++9V0zTlLa2NsnKypL6+noR6TrQXFJS0u0Adk8DJuBAMxFRtyLZ/1h+WbBbX331FebP\nn4/09HRYLB0PoH/99dfx3nvv4fjx42hra8O9996Ld955BzabTU2vrKxEXFwcNm3ahOzsbADAZ599\nhpdeegmhUAhFRUX485//DKDjlNRVq1bh0qVL0HUd7777LhISEm5qi8Vi6XZwu6Nd4VYhkmmDX18P\nm5iIaMiE22/2uExPoRBNGApERP0TSSjwimYiIlIYCkREpDAUiIhIYSgQEZHCUCAiIoWhQERECkOB\niIgUhgIRESkMhSE3BhaLpdvX+PF3jnTjiIi66PWGeDRQ7Qh3FXQgYBnephAR9YI9BSIiUhgKRESk\nMBSIiEhhKBARkcJQICIihaFARERKr6Hg8/kwf/58OBwOTJkyBW+88QYAoKWlBYsXL0Z6ejqys7Nx\n4cIFtUxJSQl0XUdmZiaOHj2qynft2gVd16HrOnbv3q3K6+vrYRgGdF1HaWnpYK4fERH1R2/P6zx3\n7pycOHFCREQCgYDcd999cuzYMSkuLpaKigoREamoqJCSkhIREdm3b5/k5uaKiIjX65Xp06eLiMjZ\ns2clNTVVAoGABAIBSU1NVc96djgc4vV6RUQkNzdX9u/f3+dnjWIUPKO5p2WIiIZKJPuYXnsKycnJ\nmDZtGgAgMTER6enp8Pv9qKqqQlFREQCgsLAQbrcbAOB2u1W5YRhob2+HaZqora1FTk4OEhMTkZiY\niCVLlqCmpgZnzpxBKBSCYRg31XXr49XORBRd+jWm8P333+PIkSN48MEH0djYiIkTJwIAkpKS0NDQ\nAADw+/2w2+1qGU3TYJom/H4/NE3rtrzz/DabDaZpDmilRo/rVzvf/AoEfhzJhhFRjOpzKFy6dAmP\nPfYYtmzZgvHjx/c4r/BB9UREo1Kf7n0UDAaxcuVKPP7448jLywMATJo0CU1NTUhKSkJjYyOsViuA\njh6Az+fD7NmzAQCmacJut0PTNNTV1ak6fT4f5s2bp+a/zjTNLj2KzsrKytR7p9MJp9PZr5UlIrqV\neTweeDyegVXS26BDKBSSoqIiefHFF7uUdx5o3rx5s7zwwgsi0jHQnJeXJyIi9fX1kp6eLiIifr9f\nUlNT5eLFi3Lx4kWZPHmynDt3TkRuHmj+8MMP+zxgglE+0MxBaCIaKpHsRyy/LBjWV199hfnz5yM9\nPR0WS8ddPdetW4dZs2YhPz8f58+fx1133YW9e/fi//7v/wAAxcXFOHjwIMaOHYu///3vyMzMBADs\n3LkTGzZsAAC8/PLL+OMf/wig45TUZ555Bm1tbVi4cCG2bt16UzssFku3h6U62hRuFSKZFj319fKn\nISLqUbj9Zo/L9BYK0YKhQETUP5GEAq9oJiIihaFAREQKQ4GIiBSGAhERKQwFIiJSGApERKQwFIiI\nSGEoEBGRwlAgIiKFoUBERApDgYiIFIYCEREpDIWo1f2jOvmYTiIaSn16yA6NhOuP6uwqELAMf1OI\nKGawp0BERApDgYiIlF5DYfXq1UhOTobD4VBlZWVl0DQNhmHAMAx89tlnatq6deuQlpYGh8OBmpoa\nVV5dXQ2Hw4G0tDSsX79elZ8+fRpz586Fw+FAQUEBgsHgYK0bERH1V2/P6/zyyy/F6/XKtGnTVFlZ\nWZls2rTppnm//vprycrKkvb2djFNU1JSUqStrU1aW1slJSVFTNOUYDAoWVlZ6pnMy5YtkwMHDoiI\nSGlpqWzevLnbdoRrKqLomcrDU9+YX6bd/Bo3bkJvf04iiiF92MXfpNeewkMPPYQJEyZ0FyY3lbnd\nbhQUFCA+Ph42mw26rqOurg51dXXQdR02mw1jxoxBfn4+3G432tvbcfjwYeTl5QEACgsL4Xa7+59s\nMeX6APTNr0Dgx5FsGBHdAiIeU3jzzTcxdepUFBYWoqWlBQDg9/uhaZqaR9M0mKYJv98Pu91+U3lj\nYyOSkpJUuc1mg2makTaJiIgGKKJTUp9//nm8+uqrADrGF0pKSlBZWTmoDetOWVmZeu90OuF0Oof8\nM4mIRguPxwOPxzOgOiIKhc6/7p999lksWLAAQEcPwOfzqWmmacJutyMUCnUp9/l8sNvtsFqtaGpq\n6jJ/557GjTqHAhERdXXjj+Xy8vJ+1xHR4aOGhgb1/sMPP4Su6wAAl8uFDz74AO3t7TBNEydPnsSs\nWbMwc+ZMnDx5En6/H8FgEHv37kVOTg7i4+MxZ84cfPTRRwCAyspKuFyuSJpERESDoNeewh/+8Ad8\n8cUXaGpqgt1uR3l5OQ4ePIjjx4+jra0N9957L9555x0AwIwZM/DII48gPT0dcXFxeOutt5CQkICE\nhARs374d2dnZCIVCKCoqQmZmJgBg69atWLVqFV555RXouo6NGzcO7RoTEVFYFunuNKIoZLFYuj3j\nyWKxoLvbQfwyNYJpo7u+UfLnJKJhEG6/2RNe0UxERApD4ZbS/Z1VeXdVIuor3iX1ltL9nVUB3l2V\niPqGPQUiIlIYCkREpDAUYgbHG4iodxxTiBkcbyCi3rGnQERECkOBiIgUhgIRESkMBSIiUhgKRESk\nMBSIiEhhKBDCXcPA6xeIYg+vUyCEu4aB1y8QxR72FIiISOk1FFavXo3k5GQ4HA5V1tLSgsWLFyM9\nPR3Z2dm4cOGCmlZSUgJd15GZmYmjR4+q8l27dkHXdei6jt27d6vy+vp6GIYBXddRWlo6WOtFg4K3\nxiCKOdKLL7/8Urxer0ybNk2VFRcXS0VFhYiIVFRUSElJiYiI7Nu3T3Jzc0VExOv1yvTp00VE5OzZ\ns5KamiqBQEACgYCkpqbK+fPnRUTE4XCI1+sVEZHc3FzZv39/t+0I11QAAkiYVyTTYq2+yD+LiKJb\nJN/TXnsKDz30ECZMmNClrKqqCkVFRQCAwsJCuN1uAIDb7VblhmGgvb0dpmmitrYWOTk5SExMRGJi\nIpYsWYKamhqcOXMGoVAIhmHcVBcREQ2/iMYUGhsbMXHiRABAUlISGhoaAAB+vx92u13Np2kaTNOE\n3++Hpmndlnee32azwTTNiFaEiIgGbtDPPpIhfHB8WVmZeu90OuF0Oofss4iIRhuPxwOPxzOgOiIK\nhUmTJqGpqQlJSUlobGyE1WoF0NED8Pl8mD17NgDANE3Y7XZomoa6ujq1vM/nw7x589T815mm2aVH\ncaPOoUBERF3d+GO5vLy833VEdPjI5XKhsrISAFBZWQmXy6XK9+zZAwDwer2Ij4+HzWbDwoULUV1d\njUAggEAggOrqaixatAh2ux1xcXHqLKU9e/aouijahT8zyWK5jWcsEY1WvY1EFxQUyN133y0JCQmi\naZrs2LFDmpubZdGiReJwOGTx4sXy448/qvmff/55SUtLE8MwpL6+XpXv2LFDpk6dKlOnTpV//OMf\nqvzrr7+WjIwMSUtLkxdeeKHfo+iI+rN7or2+4W07EQ2fSL5zll8WjHoWi6Xb8QqLxQKEeaIYEMm0\nWKtvOD+r+78hEQ2NcPvNnvCKZiIiUhgKRESkMBSIiEhhKNAw4r2UiKIdb51Nw6j7W3QDvE03UbRg\nT4GiBHsRRNGAPQWKEuxFEEUD9hRoFODjQomGC3sKNArwcaFEw4U9BSIiUhgKNIpxcJposPHwEY1i\nHJwmGmzsKdAtir0Iokiwp0C3KPYiiCLBngLFIPYiiMJhT4FiEHsRROEMqKeQkpKC9PR0GIaBWbNm\nAQBaWlqwePFipKenIzs7GxcuXFDzl5SUQNd1ZGZmqkdwAsCuXbug6zp0Xcfu3bsH0iQiIhqIgTzq\nLSUlRZqbm7uUFRcXS0VFhYiIVFRUSElJiYiI7Nu3T3Jzc0VExOv1yvTp00VE5OzZs5KamiqBQEAC\ngYCkpqbKuXPnbvqscE3FKHgEZXTXN5rbPhTbYswv0298JYQph4wbN2EgXyOiIRPJLn7AYwodn/ur\nqqoqFBUVAQAKCwvhdrsBAG63W5UbhoH29naYpona2lrk5OQgMTERiYmJWLJkCWprawfaLKIIXT+0\ndOMrGKZcEAj8ODJNJRoCAwoFi8WiDhVt27YNANDY2IiJEycCAJKSktDQ0AAA8Pv9sNvtallN02Ca\nJvx+PzRNu6mciIiG34AGmg8fPgyr1YrGxkYsWbIEDzzwQI/z39ir6K+ysjL13ul0wul0Dqg+osHR\ncTZT9xLQ0cvoaty4Cbh4sWVIW0Wxx+PxwOPxDKiOAYWC1WoFAEyaNAmPPfYYjhw5gkmTJqGpqQlJ\nSUlobGxU82iaBp/Ph9mzZwMATNOE3W6Hpmmoq6tTdfp8PsybN6/bz+scCkTRI/zZTICl22k8y4mG\nwo0/lsvLy/tdR8SHjy5fvozLly8DAH7++WdUV1dD13W4XC5UVlYCACorK+FyuQAALpcLe/bsAQB4\nvV7Ex8fDZrNh4cKFqK6uRiAQQCAQQHV1NRYtWhRps4hGCV4rQdEp4p7C+fPnkZeXB4vFgsuXL6Og\noAArVqzAgw8+iPz8fOzYsQN33XUX9u7dCwBYuXIlDh48CF3XMXbsWOzcuRMAcM8992Dt2rWqB/Hq\nq68iOTl5EFaNKJr1dK1EQtjDUTzsREPNIgM90D9MLBZLt2MSHV+e/nXde54Wa/UN52dFe33D+VmR\n1peAjkDpimFx6xs//s6wZ7qF+/uH22/2hKEwKMuM5vqG87Oivb7h/KzBrq/7sAAYGLeK3vZ14faP\n/d3F895HRLeEcNdXCAKBAMcvqM8YCkS3vEgC4zYGSYxiKBDFtEiu4A7f84gkTMaPv5MBFEV4l1Qi\n6qf+X5cB9HxWVfhleD3HcGNPgYiGSbheSU/CX88RrlfC3sXAMBSIKIqFHw8Jd4iLA+sDw8NHRHSL\niezCwHD3qQpffmue7stQIKIYEsl4SKTjJKMzTBgKREQRG+xB9/BBMlwYCkREwy5cmPR2NfvQ40Az\nEREpDAUiIlIYCkREpDAUiIhIiZpQqK6uhsPhQFpaGtavXz/SzSEiiklREQpXr17Fc889h+rqahw/\nfhz79u3D0aNHR7pZREQxJypCoa6uDrquw2azYcyYMcjPz4fb7R7pZhERxZyoCAXTNGG329W/NU2D\naZoj2CIiotgUFRevhb9MvKvly5cPcUuIiGJbVISCpmnw+Xzq3z6fr0vPAQBSU1Px6aefhqmhp1CJ\nZFqs1TecnxXt9Q3nZ8VafcP5WdFe3+B/Vnc/rlNTU3uoK0w90t+nOg+B1tZWPPDAA/jXv/4Fq9WK\nefPm4a233kJmZuZIN42IKKZERU/hN7/5DbZv347s7GyEQiEUFRUxEIiIRkBU9BSIiCg6RMXZRz2J\n5YvaVq9ejeTkZDgcDlXW0tKCxYsXIz09HdnZ2bhw4cIItnD4+Hw+zJ8/Hw6HA1OmTMEbb7wBIDa3\nR2trK2bOnAnDMHD//fdjzZo1AIDTp09j7ty5cDgcKCgoQDA4srdgHk7Xrl2DYRjqZJRY3RYpKSlI\nT0+HYRgtD5mvAAADrklEQVSYNWsWgAi+IxLFWltbJSUlRUzTlGAwKFlZWeL1eke6WcPmyy+/FK/X\nK9OmTVNlxcXFUlFRISIiFRUVUlJSMlLNG1bnzp2TEydOiIhIIBCQ++67T44dOxaz2+Py5csiIhIM\nBmX27Nny+eefy7Jly+TAgQMiIlJaWiqbN28eySYOq02bNsmqVatk+fLlIiIxuy1SUlKkubm5S1l/\nvyNRHQpffPGFLF26VP17w4YN8tprr41gi4bf6dOnu4TC5MmTpampSUREGhsbJTU1daSaNqJWrlwp\nbrc75rfHzz//LFlZWXLy5ElJSkpS5UeOHJGFCxeOYMuGj8/nk4ULF6pgbG9vj9ltkZKSor4P1/X3\nOxLVh494UdvNGhsbMXHiRABAUlISGhoaRrhFw+/777/HkSNH8OCDD8bs9giFQsjIyEBycjIWLFiA\nCRMmICkpSU232Wwx811Zs2YNNmzYgLi4jt1ZQ0NDzG4Li8WiDhVt27YNQP/3GVFx9lE4fb2ojWLH\npUuX8Nhjj2HLli0YP378SDdnxMTFxeHYsWP46aefkJ2djYyMjJFu0oj49NNPYbVaYRgGPB4PAEBi\n+NyZw4cPw2q1orGxEUuWLMEDDzzQ7zqiOhT6clFbrJk0aRKampqQlJSExsZGWK3WkW7SsAkGg1i5\nciUef/xx5OXlAYjt7QEAd9xxB5YuXYpTp06hqalJlZumCU3TRrBlw+PQoUP45JNPUFVVhdbWVly8\neBEvv/xyTG4LAOr//6RJk/DYY4/hyJEj/f6ORPXho5kzZ+LkyZPw+/0IBoPYu3cvcnJyRrpZI8rl\ncqGyshIAUFlZCZfLNcItGh4igqeffhppaWnqbBsgNrdHc3MzAoEAAODKlSuora1FRkYG5syZg48+\n+ghA7GyL119/HT6fD6dPn8b777+Phx9+GO+++25MbovLly/j8uXLAICff/4Z1dXV0HW9/9+RIRrv\nGDRVVVWi67pMnTpVXn/99ZFuzrAqKCiQu+++WxISEkTTNNmxY4c0NzfLokWLxOFwyOLFi+XHH38c\n6WYOi3/+859isVhk+vTpkpGRIRkZGfLZZ5/F5PY4fvy4ZGRkyPTp02XKlClSXl4uIiKnTp2SOXPm\nyLRp0yQ/P1/a2tpGuKXDy+PxqLOPYnFbnDp1StLT02X69Oly3333ySuvvCIi0u/vCC9eIyIiJaoP\nHxER0fBiKBARkcJQICIihaFAREQKQ4GIiBSGAhERKQwFIiJSGApERKT8P42GmqkcsevEAAAAAElF\nTkSuQmCC\n",
"text": [
"<matplotlib.figure.Figure at 0x3b95590>"
]
}
],
"prompt_number": 34
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"num_pages = page_titles.count()\n",
"num_linked_pages = number_of_inlinks.count()\n",
"print \"Existing pages: %i\\nMissing pages: %i\" % (num_pages, num_linked_pages - num_pages)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"Existing pages: 6222620\n",
"Missing pages: 10166483\n"
]
}
],
"prompt_number": 35
},
{
"cell_type": "heading",
"level": 3,
"metadata": {},
"source": [
"Counting Out-Links"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"outlink_counts = wiki.map(lambda x: (x.split(\"\\t\")[0], len(list(extract_links(x)))))"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 36
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"%time outlink_count_frequencies = outlink_counts.map(lambda x: x[1]).countByValue()"
],
"language": "python",
"metadata": {},
"outputs": [
{
"output_type": "stream",
"stream": "stdout",
"text": [
"CPU times: user 196 ms, sys: 40 ms, total: 236 ms\n",
"Wall time: 25.6 s\n"
]
}
],
"prompt_number": 37
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"pylab.hist(outlink_count_frequencies.keys(), bins=range(0, 400, 4), weights=outlink_count_frequencies.values());"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "display_data",
"png": "iVBORw0KGgoAAAANSUhEUgAAAY4AAAEACAYAAACkvpHUAAAABHNCSVQICAgIfAhkiAAAAAlwSFlz\nAAALEgAACxIB0t1+/AAAIABJREFUeJzt3XtMXGX+P/D3QLuuCUVbYKg7M1lW1gscBhi2N6o21JZQ\nRluatV2wglptYowI6R+m2fhbBY1talVS081mE223Ffxq04s1Dk7A1Ym728Ai09ri7mZ3WyxzxrXc\nrJ1auc7n9wdyBMoMHKDMAO9XQjLzzDPn+czTdt59zg2DiAiIiIjGKSLUBRAR0czC4CAiIl0YHERE\npAuDg4iIdGFwEBGRLgwOIiLSZczguHTpEjZv3oy0tDQkJSWhrq4OnZ2dyM7ORmpqKnJycnDp0iWt\nf0lJCRRFQUZGBk6dOqW1Hzx4EIqiQFEUHDp0SGtvbGyEzWaDoigoLS3V2oONQUREISRj2LRpk7z9\n9tsiItLf3y/ffvutFBcXS0VFhYiIVFRUSElJiYiIHDlyRPLy8kRExO12S1pamoiIfPXVV5KYmCg+\nn098Pp8kJibKxYsXRUTEarWK2+0WEZG8vDw5duyYiEjAMYiIKLSCrjg6Ojpw+vRpPPjggwCAiIgI\nREdHo7q6GkVFRQCAwsJCOBwOAIDD4dDabTYb+vr6oKoqamtrkZubi6ioKERFRWHdunWoqalBS0sL\n/H4/bDbbNdsKNAYREYVW0OD4z3/+g7i4OPzmN79BSkoKHn74Yfh8PrS1tSEmJgYAEBsbi9bWVgCA\n1+uFxWLR3m82m6GqKrxeL8xm86jtQ/ubTCaoqgoAAccgIqLQChocfr8fDQ0NeOaZZ9DU1IRFixbh\nxRdfDLpB4R1MiIhmtXnBXrRYLDCZTFi6dCkAYNOmTXjhhRdgNBrR3t6O2NhYtLW1wWg0AhhYSXg8\nHixfvhwAoKoqLBYLzGYz6uvrte16PB6sXLlS6z9osD8AxMXFjTrGSL/85S9x7ty5SUwBEdHckpiY\niP/+978Tfn/QFYfFYkFsbCz+/e9/AwA++ugjJCUlITc3F5WVlQCAyspK2O12AIDdbkdVVRUAwO12\nIzIyEiaTCWvWrIHT6YTP54PP54PT6cTatWthsVgQERGhnX1VVVWF3NxcbVujjTHSuXPnICJh/fP8\n88+HvAbWyTpZJ+sc/Jnsf7aDrjgA4M0338RDDz2Eq1ev4uc//zmqqqogIsjPz8f+/fuxePFiHD58\nGADwwAMP4JNPPoGiKLjhhhtw4MABAMDPfvYzPPvss9pK5LnnnkN8fDwA4MCBA3jsscfQ09ODNWvW\n4Ne//jUAoLy8fNQxiIgotMYMjrS0NDQ0NFzTXltbO2r/ffv2jdq+detWbN269Zr2X/3qV8Ou9xi0\naNGigGMQEVHo8MrxaZCVlRXqEsaFdU4t1jm1WGf4MIjIjD4NymAwYIZ/BCKiaTXZ702uOIiISBcG\nBxER6cLgICIiXRgcRESkC4ODiIh0YXAQEZEuDA4iItKFwUFERLowOIiISBcGBxER6cLgICIiXeZM\ncERHL4LBYIDBYEB09KJQl0NENGPNmZscGgwGAIP9eGNEIpq7eJNDIiKaVgwOIiLShcFBRES6MDiI\niEgXBgcREekyR4NjHk/NJSKaoDl7Oi5PzSWiuYqn4xIR0bRicBARkS4MDiIi0oXBQUREujA4iIhI\nFwYHERHpMmZwJCQkIDU1FTabDcuWLQMAdHZ2Ijs7G6mpqcjJycGlS5e0/iUlJVAUBRkZGTh16pTW\nfvDgQSiKAkVRcOjQIa29sbERNpsNiqKgtLRUaw82BhERhZCMISEhQTo6Ooa1FRcXS0VFhYiIVFRU\nSElJiYiIHDlyRPLy8kRExO12S1pamoiIfPXVV5KYmCg+n098Pp8kJibKxYsXRUTEarWK2+0WEZG8\nvDw5duxY0DFGGsdH0PoB8sPP8MdERHPJZL/3xrWrSkZcKFJdXY2ioiIAQGFhIRwOBwDA4XBo7Tab\nDX19fVBVFbW1tcjNzUVUVBSioqKwbt061NTUoKWlBX6/Hzab7ZptBRqDiIhCa8zgMBgM2i6jffv2\nAQDa2toQExMDAIiNjUVraysAwOv1wmKxaO81m81QVRVerxdms3nU9qH9TSYTVFUNOgYREYXWvLE6\n1NXVwWg0oq2tDevWrcOdd94ZtP/I1QkREc0uYwaH0WgEAMTFxWHTpk1oaGhAXFwc2tvbERsbi7a2\nNq2P2WyGx+PB8uXLAQCqqsJiscBsNqO+vl7bpsfjwcqVK7X+gwb7D4432hijKSsr0x5nZWUhKysL\n0dGL4PN9o2MqiIhmJ5fLBZfLNXUbDHYA5LvvvpPvvvtORESuXLkiq1atkhMnTgw7cP3aa6/J008/\nLSIDB8c3btwoIiKNjY2SmpoqIiJer1cSExPl8uXLcvnyZbn11lvl66+/FpFrD44fPXpURCTgGCMF\n+ggYdgCcB8eJiAZN9nsv6N1xm5ubsXHjRhgMBly9ehUFBQV44YUX0NnZifz8fFy8eBGLFy/G4cOH\ncfPNNwMAiouL8cknn+CGG27AG2+8gYyMDADAgQMHsGfPHgDAjh078MgjjwAYOB1327Zt6OnpwZo1\na/D6668DQNAxhgp0l8fhd8MFRt4Rl3fHJaK5arJ3x521t1VncBARjY63VSciomnF4CAiIl0YHERE\npAuDg4iIdGFwEBGRLgwOIiLShcFBRES6MDiIiEgXBgcREenC4CAiIl0YHEREpAuDg4iIdGFwEBGR\nLgwOIiLShcFBRES6MDiIiEgXBgcREenC4CAiIl0YHEREpAuDg4iIdGFwEBGRLgwOIiLShcGBeTAY\nDNpPdPSiUBdERBTWDCIioS5iMgwGA0b7CAaDAcDQ9qHPAz0eeD7Dp4SIKKhA35vjxRUHERHpwuAg\nIiJdGBxERKQLg4OIiHRhcBARkS7jCo7+/n7YbDasX78eANDc3IzMzExYrVYUFBSgt7cXANDd3Y38\n/HxYrVbcdddduHDhgraNXbt2ITk5GVarFTU1NVq70+mE1WpFcnIydu/erbUHGoOIiEJrXMGxd+9e\nJCcn/3CKK1BSUoIdO3bg7NmzWLx4Mfbt2wcA2LdvH2655RacPXsWzzzzDEpKSgAAjY2NOHbsGM6e\nPQun04knnngCvb296O7uxpNPPgmn04kzZ87gyJEjOHXqVNAxiIgotMYMDlVVUV1djW3btkFE0N/f\nj7q6OmzcuBEAUFhYCIfDAQCorq5GUVERAGDDhg04efIk/H4/HA4HCgoKEBkZCZPJBEVRUF9fj/r6\neiiKApPJhHnz5iE/Px8OhwN9fX0BxyAiotAaMzi2b9+OPXv2ICJioGtraytiY2O1100mE1RVBTAQ\nMhaLZWDDERGIiYlBa2srvF4vzGaz9h6z2QxVVeH1erX+Q9vb2toCjkFERKE1L9iLH3zwAYxGI2w2\nG1wuFwCE5VXVZWVl2uOsrCxkZWWFrBYionDjcrm07/CpEDQ4Tp48iffffx/V1dXo6urC5cuXsWPH\nDrS3t2t9VFXVVhNmsxktLS0wGo3w+/3o6OhAXFwczGYzPB7PsPdYLBb4/f5h7R6PBxaLBUajMeAY\noxkaHERENNzI/1CXl5dPantBd1Xt3LkTHo8Hzc3NeOedd3DvvffirbfewooVK/Dee+8BACorK2G3\n2wEAdrsdlZWVAIATJ04gMzMTkZGRsNvtePfdd9HX1wdVVdHU1IRly5Zh6dKlaGpqgtfrRW9vLw4f\nPozc3FxERkYGHIOIiEJMxsnlcsn69etFROT8+fOyYsUKSUlJkfz8fOnp6RERka6uLtm8ebOkpKRI\nZmamNDc3a+9/6aWXJCkpSRRFEafTqbVXV1eLoiiSlJQkO3fu1NoDjTFSoI8AQAAZ8oNxPA68PSKi\n2WKy33O8Oy7vjktEcwzvjjtEdPQi7fdqEBHR9TGrVhzDVxlccRARjYYrDiIimlYMDiIi0oXBQURE\nujA4iIhIFwYHERHpwuAgIiJdGBzXmKddCxIdvSjUxRARhZ2gNzmcm/oweF2Hz8cLCYmIRuKKg4iI\ndGFwEBGRLgwOIiLShcFBRES6MDiIiEgXBgcREenC4CAiIl0YHEREpAuDg4iIdGFwEBGRLgwOIiLS\nhcFBRES6MDiIiEgXBgcREenC4CAiIl0YHEREpAuDg4iIdGFwEBGRLkGDo6urC0uXLoXNZsPtt9+O\n7du3AwCam5uRmZkJq9WKgoIC9Pb2AgC6u7uRn58Pq9WKu+66CxcuXNC2tWvXLiQnJ8NqtaKmpkZr\ndzqdsFqtSE5Oxu7du7X2QGNML/7+cSKia8gYrl69KiIivb29snz5cvn444/l/vvvl+PHj4uISGlp\nqbz22msiIvLKK69IaWmpiIgcP35cNmzYICIin332mSxZskT6+vpEVVVJSEiQnp4e6erqkoSEBFFV\nVXp7e2XJkiXidrtFRAKOMdLQjwBAAPnhZ+jjYK+Nvx8R0Www2e+zMXdV3XjjjQCAnp4e9Pf3w2g0\noq6uDhs3bgQAFBYWwuFwAACqq6tRVFQEANiwYQNOnjwJv98Ph8OBgoICREZGwmQyQVEU1NfXo76+\nHoqiwGQyYd68ecjPz4fD4UBfX1/AMYiIKLTGDA6/34/09HTEx8dj9erVWLhwIWJjY7XXTSYTVFUF\nAKiqCovFMrDhiAjExMSgtbUVXq8XZrNZe4/ZbIaqqvB6vVr/oe1tbW0BxyAiotCaN1aHiIgInD59\nGt9++y1ycnKQnp4+HXXpUlZWNuSZC0BWSOogIgpHLpcLLpdryrY3ZnAMuummm3Dffffh/PnzaG9v\n19pVVdVWE2azGS0tLTAajfD7/ejo6EBcXBzMZjM8Hs+w91gsFvj9/mHtHo8HFosFRqMx4BijGQyO\n8vJyMDSIiIbLyspCVlaW9nzgu3Ligu6q6ujogM/nAwB8//33qK2tRXp6OlasWIH33nsPAFBZWQm7\n3Q4AsNvtqKysBACcOHECmZmZiIyMhN1ux7vvvou+vj6oqoqmpiYsW7YMS5cuRVNTE7xeL3p7e3H4\n8GHk5uYiMjIy4BhERBRahh+OsI/q7NmzePjhhyEi6OrqwpYtW/Dcc8+hubkZW7ZswZUrV6AoCt56\n6y3Mnz8f3d3dKCoqwj//+U8sWLAAb7/9NhISEgAAO3fuRGVlJSIiIvDqq68iJycHAPDhhx/imWee\ngd/vR1FREX77298CQMAxrvkABgMGP4LBYAAw+HGGPh75fGL9gkwVEdGMMfR7c0LvDxYcMwGDg4hI\nn8kGB68cJyIiXRgcRESkC4ODiIh0YXAQEZEuDA4iItKFwTFuP94pl3fLJaK5bNxXjlMfhp626/MZ\nQlcKEVEIccVBRES6MDiIiEgXBgcREenC4CAiIl0YHEREpAuDg4iIdGFwEBGRLgwOIiLShcFBRES6\nMDiIiEgXBgcREenC4JiwebzhIRHNSbzJ4YT9eNND3vCQiOYSrjiIiEgXBgcREenC4CAiIl0YHERE\npAuDg4iIdGFwEBGRLgwOIiLShcFBRES6BA0Oj8eDVatWwWq14o477sDLL78MAOjs7ER2djZSU1OR\nk5ODS5cuae8pKSmBoijIyMjAqVOntPaDBw9CURQoioJDhw5p7Y2NjbDZbFAUBaWlpVp7sDGIiCiE\nJIivv/5azp49KyIiPp9PbrvtNjl9+rQUFxdLRUWFiIhUVFRISUmJiIgcOXJE8vLyRETE7XZLWlqa\niIh89dVXkpiYKD6fT3w+nyQmJsrFixdFRMRqtYrb7RYRkby8PDl27JiISMAxRhr6EQAIID/8DH0c\n7LWp6UdENFNM9jsr6IojPj4eKSkpAICoqCikpqbC6/WiuroaRUVFAIDCwkI4HA4AgMPh0NptNhv6\n+vqgqipqa2uRm5uLqKgoREVFYd26daipqUFLSwv8fj9sNts12wo0BhERhda4j3F8+eWXaGhowN13\n3422tjbExMQAAGJjY9Ha2goA8Hq9sFgs2nvMZjNUVYXX64XZbB61fWh/k8kEVVUBIOAYREQUWuO6\nyeGVK1ewadMm7N27F9HR0UH7DqyCpldZWdmQZy4AWdNcwcCdcgFgwYKFuHy5c5rHJyIKzOVyweVy\nTdn2xgyO3t5ePPDAA3jooYewceNGAEBcXBza29sRGxuLtrY2GI1GAAMrCY/Hg+XLlwMAVFWFxWKB\n2WxGfX29tk2Px4OVK1dq/QcN9g82xmgGg6O8vBzTHxoA75RLROEsKysLWVlZ2vOB78qJC7qrSkTw\n+OOPIzk5Gdu3b9fa7XY7KisrAQCVlZWw2+1ae1VVFQDA7XYjMjISJpMJa9asgdPphM/ng8/ng9Pp\nxNq1a2GxWBAREaGdfVVVVYXc3NygYxARUWgZJMi+pb/+9a9YtWoVUlNTtV0xu3btwrJly5Cfn4+L\nFy9i8eLFOHz4MG6++WYAQHFxMT755BPccMMNeOONN5CRkQEAOHDgAPbs2QMA2LFjBx555BEAA6fj\nbtu2DT09PVizZg1ef/11AAOn4wYaY9gHMBi03WMDNQ5+nKGPRz6/vv1CsbuOiGi8hn5vTuj9wYJj\nJmBwEBHpM9ng4JXjRESkC4ODiIh0YXAQEZEuDA4iItKFwUFERLowOKbcwFXkgz/R0YtCXRAR0ZQa\n1y1HSI8fryIHeCU5Ec0+XHEQEZEuDA4iItKFwUFERLrMimMcOTmbQ10CEdGcMSvuVQUcBvB/AI4j\nHO5VNbLfDJ9iIppleK8qAMBmGAzJoS6CiGhOmCXBQURE04XBcd3N48WARDSrzIqD4+GNv1aWiGYX\nrjiIiEgXBgcREenC4CAiIl0YHEREpAuDg4iIdGFwTCv+rg4imvl4Ou604u/qIKKZjysOIiLShcFB\nRES6MDhCircjIaKZh8c4Qoq3IyGimYcrDiIi0mXM4HjssccQHx8Pq9WqtXV2diI7OxupqanIycnB\npUuXtNdKSkqgKAoyMjJw6tQprf3gwYNQFAWKouDQoUNae2NjI2w2GxRFQWlp6bjGICKi0BkzOLZu\n3Qqn0zms7fnnn8d9992HM2fOIDc3F88//zwA4OjRo2hpacEXX3yBN998E1u3bgUA/O9//8OLL76I\n+vp61NfX44UXXkBra6u2/f379+OLL77AhQsXcPz48aBjEBFRaI0ZHPfccw8WLlw4rK26uhpFRUUA\ngMLCQjgcDgCAw+HQ2m02G/r6+qCqKmpra5Gbm4uoqChERUVh3bp1qKmpQUtLC/x+P2w22zXbCjQG\nERGF1oSOcbS1tSEmJgYAEBsbq60evF4vLBaL1s9sNkNVVXi9XpjN5lHbh/Y3mUxQVTXoGLMXz7Ai\noplhys+qmswvQJ/beIYVEc0MEwqOuLg4tLe3IzY2Fm1tbTAajQAGVhIejwfLly8HAKiqCovFArPZ\njPr6eu39Ho8HK1eu1PoPGuwfbIzRlUHk0x8euwBkTeRjERHNSi6XCy6Xa8q2N6FdVXa7HZWVlQCA\nyspK2O12rb2qqgoA4Ha7ERkZCZPJhDVr1sDpdMLn88Hn88HpdGLt2rWwWCyIiIjQzr6qqqpCbm5u\n0DFGVwaDYdUPj7Mm8pHCDG+GSERTJysrC2VlZdrPpMkYCgoK5JZbbpH58+eL2WyW/fv3S0dHh6xd\nu1asVqtkZ2fLN998o/V/6qmnJDk5WWw2mzQ2Nmrt+/fvl6SkJElKSpI//elPWvtnn30m6enpkpyc\nLE8//bTWHmyMoQAIIGIwPKs9HvgZ+njk83Dop28bRERTZbLfKYYfNjJjGQwGAAKD4f9B5CX8ePdZ\nw5DHI58Hejyd/fRtY4b/MRFRGDEYJvedwivHiYhIFwbHjMBTdYkofPAmhzMCT9UlovDBFQcREenC\n4JhxuNuKiEKLu6pmHO62IqLQ4opjRuOFgkQ0/bjimNF+XH0AXIEQ0fTgioOIiHRhcMwqPHBORNcf\ng2NWGdx1JfD5fAwRIroueIxj1uLZV0R0fXDFQUREujA45gSetktEU4fBMSf8eOyDxz+IaLJ4jGNO\n4vEPIpo4rjjmvKG7sX7CXVpENCauOOa8oVefD//Ng1yNENFouOKgIHhBIRFdi8FBQfCCQiK6FoOD\nxmn0EBl5bIShQjT78RgHTcDwu/IOPTbC4yJEsx9XHDTFAp+lxZUJ0ezA4KApNvRiw14MvfBw6HMe\nMyGauRgcFCLjO2bCa0uIwg+Dg8LA8FuiDF+pDF+1DA8Z7voiCgUGB80wo+8KG++qhQFDNHkMDpol\nxrdqYcAQTV7YB4fT6YTVakVycjJ2794d6nJoxpvagAl25hiP1dBsFdbB0d3djSeffBJOpxNnzpzB\nkSNHcOrUqVCXNQGuUBcwy7imYYzxHncJfObYRI7VjPcU5ujoRVMWRC6Xa8LvnU6sM3yEdXDU19dD\nURSYTCbMmzcP+fn5cDgcoS5rAlyhLmCWcYW6gCkQ6LTl8Z3C7PN9g/EF0diroNWrV+vabTc8tMbX\nbypWWDPlC3mm1DkZYR0cqqrCYrFoz81mM1RVDWFFRKE0NGyCvaZ3FfR80H4jd9sND63x9ZtMsA3+\nlJe/OKW7Cyfy2nhCtLy8fNbvmgzrW44YDOO7fUV09Hp0d/8L3d3XuSCiOWm0W8zo7Rf49v3Dnwd6\nPJ39Ar/m880f5XtpZL8yAOXDtjf8ffMxELgjHwd7bfL9FixYiMuXOzEVwjo4zGYzPB6P9tzj8Qxb\ngQBAYmIizp37YEiLIcDjYK9d737lP/yEU00ztR8QeD7DvfZQztlY2xhrPsP9M07nnI003rEG9QZ4\nHOy1yffz+b7RwisxMTFAbeNjEJHR1r1hoaurC3feeSf+9re/wWg0YuXKlfjjH/+IjIyMUJdGRDRn\nhfWK46c//Sn+8Ic/ICcnB36/H0VFRQwNIqIQC+sVBxERhZ+wPqsqmHC+MDAhIQGpqamw2WxYtmwZ\nAKCzsxPZ2dlITU1FTk4OLl26NO11PfbYY4iPj4fVatXagtVVUlICRVGQkZExrdfPjFZnWVkZzGYz\nbDYbbDYbPvzwQ+21Xbt2ITk5GVarFTU1NdNSo8fjwapVq2C1WnHHHXfg5ZdfBhB+8xmoznCbz66u\nLixduhQ2mw233347tm/fDgBobm5GZmYmrFYrCgoK0Ns7sM++u7sb+fn5sFqtuOuuu3DhwoWQ1vno\no4/i1ltv1ebz888/BwCISMj+HQFAf38/bDYb1q9fD2AK51NmoK6uLklISBBVVaW3t1eWLFkibrc7\n1GVpEhISpKOjY1hbcXGxVFRUiIhIRUWFlJSUTHtdn376qbjdbklJSRmzriNHjkheXp6IiLjdbklL\nSwtpnWVlZfLqq69e0/ezzz6TJUuWSF9fn6iqKgkJCdLd3X3da/z666/l7NmzIiLi8/nktttuk9On\nT4fdfAaqM9zmU0Tk6tWrIiLS29sry5cvl48//ljuv/9+OX78uIiIlJaWymuvvSYiIq+88oqUlpaK\niMjx48dlw4YN01JjoDofffRROXr06DV9Q/nvSETk1VdflS1btsj69etFRKZsPmfkimMmXBgoI/YA\nVldXo6ioCABQWFgYknrvueceLFy4cFx1ORwOrd1ms6Gvr2/arqEZrU7g2jkFBuosKChAZGQkTCYT\nFEXB3//+9+teY3x8PFJSUgAAUVFRSE1NhdfrDbv5DFQnEF7zCQA33ngjAKCnpwf9/f0wGo2oq6vD\nxo0bAQyfz6HzvGHDBpw8eXLUzzNddQKjz+fQOqf735Gqqqiursa2bdsgIujv75+y+ZyRwRHuFwYa\nDAZtd8W+ffsAAG1tbYiJiQEAxMbGorW1NZQlagLV5fV6w26Of//73yMpKQmFhYXo7Bw4H93r9cJs\nNmt9QlHnl19+iYaGBtx9991hPZ+Ddd5zzz0Awm8+/X4/0tPTER8fj9WrV2PhwoWIjY3VXjeZTFot\nQ78DIiIiEBMTM23/pkbWqSgKAODZZ59FUlISiouL0f3DRWWh/K7avn079uzZg4iIga/51tbWKZvP\nGRkc470wMFTq6urgdrvx5z//GQcOHMBHH30U6pImZOT/OEI570899RTOnTuHf/zjH0hMTERJSUnI\nahnqypUr2LRpE/bu3Yvo6OigfUM5n1euXMHmzZuxd+9eLFiwICznMyIiAqdPn4aqqvj000/D9tYd\no9W5e/du/Otf/8Lnn3+O77//Hi+++KLWPxR/7h988AGMRiNsNps2/lSuyGZkcIznwsBQGly6xsXF\nYdOmTWhoaEBcXBza29sBDPwvf7BPqAWqa+Qcq6o67H+i0y02Nla7fcMTTzyBhoYGAKPXOV1/F3p7\ne/HAAw/goYce0pb/4Tifg3Vu2bJFqzMc53PQTTfdhPvuuw/nz5/X5nKwlsE5M5vNaGlpATCwAujo\n6EBcXFxI6qyrq9P+nH/yk5/g8ccfDzqf0/HnfvLkSbz//vv4xS9+gQcffBAff/wxduzYMWXzOSOD\nY+nSpWhqaoLX60Vvby8OHz6M3NzcUJcFALh69SquXr0KAPjuu+/gdDqhKArsdjsqKysBAJWVlbDb\n7aEsUxOoLrvdjqqqKgCA2+3W9nmHytBl89GjR7XdA3a7He+++66277ipqUk7k+16EhE8/vjjSE5O\n1s6sGawnnOYzUJ3hNp8dHR3w+XwAgO+//x61tbVIT0/HihUr8N577wG4dj4H5/nEiRPIzMzUdslM\nd51Wq1WbTxHBsWPHhs1nKP7cd+7cCY/Hg+bmZrzzzju499578dZbb03dfE7lEfzpVF1dLYqiSFJS\nkuzcuTPU5WjOnz8vqampkpaWJrfddpv87ne/ExGRjo4OWbt2rVitVsnOzpZvvvlm2msrKCiQW265\nRebPny9ms1n2798ftK6nnnpKkpOTxWazSWNjY8jqfPPNN6WwsFBSU1PlzjvvlJycHFFVVev/0ksv\nSVJSkiiKIk6nc1pq/Mtf/iIGg0HS0tIkPT1d0tPT5cMPPwy7+Rytzurq6rCbzzNnzkh6erqkpaXJ\nHXfcIeXl5SIy8O9pxYoVkpKSIvn5+dLT0yMiA2dWbt68WVJSUiQzM1Oam5tDWufq1aslLS1Nbr/9\ndsnPz5fHxsz5AAAAT0lEQVRvv/1We0+o/h0Ncrlc2llVUzWfvACQiIh0mZG7qoiIKHQYHEREpAuD\ng4iIdGFwEBGRLgwOIiLShcFBRES6MDiIiEgXBgcREeny/wE017Lg98NrOQAAAABJRU5ErkJggg==\n",
"text": [
"<matplotlib.figure.Figure at 0x3b7d190>"
]
}
],
"prompt_number": 38
},
{
"cell_type": "heading",
"level": 4,
"metadata": {},
"source": [
"Pages with many links:"
]
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"prolific_linkers = outlink_counts.filter(lambda (title, count): count > 400).collectAsMap()"
],
"language": "python",
"metadata": {},
"outputs": [],
"prompt_number": 39
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"len(prolific_linkers)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"metadata": {},
"output_type": "pyout",
"prompt_number": 40,
"text": [
"115260"
]
}
],
"prompt_number": 40
},
{
"cell_type": "code",
"collapsed": false,
"input": [
"import operator\n",
"top_linkers = sorted(prolific_linkers.iteritems(), key=operator.itemgetter(1), reverse=True)[:500]\n",
"wiki_table(top_linkers)"
],
"language": "python",
"metadata": {},
"outputs": [
{
"html": [
"<table><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ISO_639-3_codes\" target=\"_BLANK\">List_of_ISO_639-3_codes</a></td><td>7715</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Places_in_Afghanistan\" target=\"_BLANK\">Places_in_Afghanistan</a></td><td>7246</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Afghanistan\" target=\"_BLANK\">List_of_places_in_Afghanistan</a></td><td>7246</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World_War_I_flying_aces\" target=\"_BLANK\">World_War_I_flying_aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_I_flying_aces_by_number_of_victories\" target=\"_BLANK\">List_of_World_War_I_flying_aces_by_number_of_victories</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_One_flying_aces\" target=\"_BLANK\">List_of_World_War_One_flying_aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_I_Flying_Aces\" target=\"_BLANK\">List_of_World_War_I_Flying_Aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_I_air_aces\" target=\"_BLANK\">List_of_World_War_I_air_aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_One_Flying_Aces\" target=\"_BLANK\">List_of_World_War_One_Flying_Aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_I_flying_aces_by_nationality\" target=\"_BLANK\">List_of_World_War_I_flying_aces_by_nationality</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Wwi_aces\" target=\"_BLANK\">Wwi_aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_I_flying_aces\" target=\"_BLANK\">List_of_World_War_I_flying_aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_wwi_aces\" target=\"_BLANK\">List_of_wwi_aces</a></td><td>6912</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Flora_of_New_South_Wales\" target=\"_BLANK\">Flora_of_New_South_Wales</a></td><td>6841</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_College_Athletic_Programs_by_US_State\" target=\"_BLANK\">List_of_College_Athletic_Programs_by_US_State</a></td><td>6583</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_college_athletic_programs_by_U.S._State\" target=\"_BLANK\">List_of_college_athletic_programs_by_U.S._State</a></td><td>6583</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_college_athletic_programs_by_U.S._state\" target=\"_BLANK\">List_of_college_athletic_programs_by_U.S._state</a></td><td>6583</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_College_Athletic_Programs_by_State\" target=\"_BLANK\">List_of_College_Athletic_Programs_by_State</a></td><td>6583</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Tamil_films_of_the_1980s\" target=\"_BLANK\">Tamil_films_of_the_1980s</a></td><td>6572</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Tamil_films:_1980s\" target=\"_BLANK\">List_of_Tamil_films:_1980s</a></td><td>6572</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Singapore-related_topics\" target=\"_BLANK\">List_of_Singapore-related_topics</a></td><td>6347</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World_Canoeing_Championship\" target=\"_BLANK\">World_Canoeing_Championship</a></td><td>6333</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/ICF_Canoe_Sprint_World_Championships\" target=\"_BLANK\">ICF_Canoe_Sprint_World_Championships</a></td><td>6333</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Flatwater_Racing_World_Championships\" target=\"_BLANK\">Flatwater_Racing_World_Championships</a></td><td>6333</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/ICF_Flatwater_Racing_World_Championships\" target=\"_BLANK\">ICF_Flatwater_Racing_World_Championships</a></td><td>6333</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Area_codes_in_germany\" target=\"_BLANK\">Area_codes_in_germany</a></td><td>6229</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Area_codes_in_Germany\" target=\"_BLANK\">Area_codes_in_Germany</a></td><td>6229</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_endangered_animal_species\" target=\"_BLANK\">List_of_endangered_animal_species</a></td><td>6145</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_extant_animal_genera_represented_in_the_fossil_record\" target=\"_BLANK\">List_of_extant_animal_genera_represented_in_the_fossil_record</a></td><td>5965</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Euphorbia_species\" target=\"_BLANK\">List_of_Euphorbia_species</a></td><td>5836</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_trilobites\" target=\"_BLANK\">List_of_trilobites</a></td><td>5816</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_AM_radio_stations_in_the_United_States_by_call_sign_(initial_letter_W)\" target=\"_BLANK\">List_of_AM_radio_stations_in_the_United_States_by_call_sign_(initial_letter_W)</a></td><td>5756</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Episcopal_Bishops_(U.S.)\" target=\"_BLANK\">List_of_Episcopal_Bishops_(U.S.)</a></td><td>5748</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Episcopal_bishops_(U.S.)\" target=\"_BLANK\">List_of_Episcopal_bishops_(U.S.)</a></td><td>5748</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Succession_of_Bishops_of_the_Episcopal_Church_in_the_United_States_of_America\" target=\"_BLANK\">Succession_of_Bishops_of_the_Episcopal_Church_in_the_United_States_of_America</a></td><td>5748</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Succession_of_Bishops_of_the_Episcopal_Church_in_the_United_States\" target=\"_BLANK\">Succession_of_Bishops_of_the_Episcopal_Church_in_the_United_States</a></td><td>5748</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Succession_of_Bishops_of_The_Episcopal_Church_(U.S.)\" target=\"_BLANK\">Succession_of_Bishops_of_The_Episcopal_Church_(U.S.)</a></td><td>5748</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_foreign_Ligue_1_players\" target=\"_BLANK\">List_of_foreign_Ligue_1_players</a></td><td>5629</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_settlements_in_Serbia_(alphabetic)\" target=\"_BLANK\">List_of_settlements_in_Serbia_(alphabetic)</a></td><td>5583</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Landmarked_building\" target=\"_BLANK\">Landmarked_building</a></td><td>5573</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Landmarks\" target=\"_BLANK\">Landmarks</a></td><td>5573</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Landmark\" target=\"_BLANK\">Landmark</a></td><td>5573</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Landmark_building\" target=\"_BLANK\">Landmark_building</a></td><td>5573</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_municipalities_of_Brazil\" target=\"_BLANK\">List_of_municipalities_of_Brazil</a></td><td>5572</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Slovakian_municipalities\" target=\"_BLANK\">Slovakian_municipalities</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Places_in_Slovakia\" target=\"_BLANK\">Places_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Municipalities_of_Slovakia\" target=\"_BLANK\">Municipalities_of_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Slovakian_villages\" target=\"_BLANK\">List_of_Slovakian_villages</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_municipalities_in_Slovakia\" target=\"_BLANK\">List_of_municipalities_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Slovakian_villages\" target=\"_BLANK\">Slovakian_villages</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_municipalities_and_towns_in_Slovakia\" target=\"_BLANK\">List_of_municipalities_and_towns_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_villages_and_municipalities_in_Slovakia\" target=\"_BLANK\">List_of_villages_and_municipalities_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Villages_in_Slovakia\" target=\"_BLANK\">Villages_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Slovakia\" target=\"_BLANK\">List_of_places_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_villages_in_Slovakia\" target=\"_BLANK\">List_of_villages_in_Slovakia</a></td><td>5415</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_India-related_articles\" target=\"_BLANK\">List_of_India-related_articles</a></td><td>5381</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_India-related_topics\" target=\"_BLANK\">List_of_India-related_topics</a></td><td>5381</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_India_related_topics\" target=\"_BLANK\">List_of_India_related_topics</a></td><td>5381</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Miscellaneous_India_topics\" target=\"_BLANK\">Miscellaneous_India_topics</a></td><td>5381</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_sportspeople_by_nickname\" target=\"_BLANK\">List_of_sportspeople_by_nickname</a></td><td>5321</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_athletes_by_nickname\" target=\"_BLANK\">List_of_athletes_by_nickname</a></td><td>5321</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_years\" target=\"_BLANK\">List_of_years</a></td><td>5314</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Playstation_games\" target=\"_BLANK\">Playstation_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Playstation_video_games\" target=\"_BLANK\">Playstation_video_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Popstar_Maker\" target=\"_BLANK\">Popstar_Maker</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_psx_games\" target=\"_BLANK\">List_of_psx_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_PlayStation_1_games\" target=\"_BLANK\">List_of_PlayStation_1_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sony_psx_games\" target=\"_BLANK\">Sony_psx_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sony_playstation_games\" target=\"_BLANK\">Sony_playstation_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sony_ps1_games\" target=\"_BLANK\">Sony_ps1_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ps1_games\" target=\"_BLANK\">List_of_ps1_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Ps1_games\" target=\"_BLANK\">Ps1_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_playstation_1_games\" target=\"_BLANK\">List_of_playstation_1_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sony_playstation_video_games\" target=\"_BLANK\">Sony_playstation_video_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Psx_games\" target=\"_BLANK\">Psx_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Playstation_games\" target=\"_BLANK\">List_of_Playstation_games</a></td><td>5153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Chronology_of_the_Quorum_of_the_Twelve_Apostles_(LDS_Church)\" target=\"_BLANK\">Chronology_of_the_Quorum_of_the_Twelve_Apostles_(LDS_Church)</a></td><td>5082</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Places_in_Serbia_(N-Z)\" target=\"_BLANK\">List_of_Places_in_Serbia_(N-Z)</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_settlements_in_Serbia\" target=\"_BLANK\">List_of_settlements_in_Serbia</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_cities,_towns_and_villages_in_Serbia\" target=\"_BLANK\">List_of_cities,_towns_and_villages_in_Serbia</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_villages_in_Serbia\" target=\"_BLANK\">List_of_villages_in_Serbia</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Serbia_(A-M)\" target=\"_BLANK\">List_of_places_in_Serbia_(A-M)</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Serbia_(N-Z)\" target=\"_BLANK\">List_of_places_in_Serbia_(N-Z)</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Serbia\" target=\"_BLANK\">List_of_places_in_Serbia</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_cities,_towns_and_villages_in_Central_Serbia\" target=\"_BLANK\">List_of_cities,_towns_and_villages_in_Central_Serbia</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Places_in_Serbia_(A-M)\" target=\"_BLANK\">List_of_Places_in_Serbia_(A-M)</a></td><td>4865</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Thailand-related_topics\" target=\"_BLANK\">List_of_Thailand-related_topics</a></td><td>4741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Thailand-related_articles\" target=\"_BLANK\">List_of_Thailand-related_articles</a></td><td>4741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Animated_feature_film\" target=\"_BLANK\">Animated_feature_film</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Animated_Feature_Film\" target=\"_BLANK\">List_of_Animated_Feature_Film</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Animated_Features\" target=\"_BLANK\">List_of_Animated_Features</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_animated_feature_films\" target=\"_BLANK\">List_of_animated_feature_films</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_animated_movies\" target=\"_BLANK\">List_of_animated_movies</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_animated_feature-length_films\" target=\"_BLANK\">List_of_animated_feature-length_films</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_animated_features\" target=\"_BLANK\">List_of_animated_features</a></td><td>4665</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_postal_codes_in_Portugal\" target=\"_BLANK\">List_of_postal_codes_in_Portugal</a></td><td>4598</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ps2_games\" target=\"_BLANK\">List_of_ps2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Playstation_2_games\" target=\"_BLANK\">Playstation_2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sony_playstation_2_games\" target=\"_BLANK\">Sony_playstation_2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Playstation_2_games\" target=\"_BLANK\">List_of_Playstation_2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Playstation_2_Video_Games\" target=\"_BLANK\">Playstation_2_Video_Games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Playstation_2_Games\" target=\"_BLANK\">List_of_Playstation_2_Games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Ps2_video_games\" target=\"_BLANK\">Ps2_video_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sony_ps2_games\" target=\"_BLANK\">Sony_ps2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/PS2_games\" target=\"_BLANK\">PS2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_PlayStation_2_games\" target=\"_BLANK\">List_of_PlayStation_2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_PS2_games\" target=\"_BLANK\">List_of_PS2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Ps2_games\" target=\"_BLANK\">Ps2_games</a></td><td>4570</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/United_States_Congressional_Delegations_from_New_York\" target=\"_BLANK\">United_States_Congressional_Delegations_from_New_York</a></td><td>4549</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/United_States_congressional_delegations_from_New_york\" target=\"_BLANK\">United_States_congressional_delegations_from_New_york</a></td><td>4549</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/U.S._Congressional_Delegations_from_New_York\" target=\"_BLANK\">U.S._Congressional_Delegations_from_New_York</a></td><td>4549</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/US_Congressional_Delegations_from_New_York\" target=\"_BLANK\">US_Congressional_Delegations_from_New_York</a></td><td>4549</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/United_States_congressional_delegations_from_New_York\" target=\"_BLANK\">United_States_congressional_delegations_from_New_York</a></td><td>4549</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/NYCongDel\" target=\"_BLANK\">NYCongDel</a></td><td>4549</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_stadia\" target=\"_BLANK\">List_of_stadia</a></td><td>4529</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_stadiums\" target=\"_BLANK\">List_of_stadiums</a></td><td>4529</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Denmark_national_football_team_fixtures\" target=\"_BLANK\">List_of_Denmark_national_football_team_fixtures</a></td><td>4495</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Denmark_national_football_team_matches\" target=\"_BLANK\">List_of_Denmark_national_football_team_matches</a></td><td>4495</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Denmark_national_football_team_results\" target=\"_BLANK\">List_of_Denmark_national_football_team_results</a></td><td>4495</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Brachiopod_Genera\" target=\"_BLANK\">List_of_Brachiopod_Genera</a></td><td>4413</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_brachiopod_genera\" target=\"_BLANK\">List_of_brachiopod_genera</a></td><td>4413</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_DOS_games\" target=\"_BLANK\">List_of_DOS_games</a></td><td>4400</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_DOS_computer_games\" target=\"_BLANK\">List_of_DOS_computer_games</a></td><td>4400</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_MS-DOS_games\" target=\"_BLANK\">List_of_MS-DOS_games</a></td><td>4400</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_MS-DOS_video_games\" target=\"_BLANK\">List_of_MS-DOS_video_games</a></td><td>4400</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_DOS_video_games\" target=\"_BLANK\">List_of_DOS_video_games</a></td><td>4400</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_MS-DOS_computer_games\" target=\"_BLANK\">List_of_MS-DOS_computer_games</a></td><td>4400</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Members_of_the_European_Parliament_2004\u20132009\" target=\"_BLANK\">List_of_Members_of_the_European_Parliament_2004\u20132009</a></td><td>4393</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_members_of_the_European_Parliament_2004\u20132009\" target=\"_BLANK\">List_of_members_of_the_European_Parliament_2004\u20132009</a></td><td>4393</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Members_of_the_European_Parliament_2004-2009\" target=\"_BLANK\">List_of_Members_of_the_European_Parliament_2004-2009</a></td><td>4393</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Village_Development_Committee\" target=\"_BLANK\">Village_Development_Committee</a></td><td>4268</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_banks\" target=\"_BLANK\">List_of_banks</a></td><td>4268</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Village_Development_Committees\" target=\"_BLANK\">Village_Development_Committees</a></td><td>4268</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Vdc_nepal\" target=\"_BLANK\">Vdc_nepal</a></td><td>4268</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Village_Development_Committee_(Nepal)\" target=\"_BLANK\">Village_Development_Committee_(Nepal)</a></td><td>4268</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_Series_starting_pitchers\" target=\"_BLANK\">List_of_World_Series_starting_pitchers</a></td><td>4242</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ship_launches_in_1944\" target=\"_BLANK\">List_of_ship_launches_in_1944</a></td><td>4200</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Meltsch_\u00fcber_Troppau\" target=\"_BLANK\">Meltsch_\u00fcber_Troppau</a></td><td>4148</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_German_exonyms_for_places_in_the_Czech_Republic\" target=\"_BLANK\">List_of_German_exonyms_for_places_in_the_Czech_Republic</a></td><td>4148</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Meltsch_ueber_Troppau\" target=\"_BLANK\">Meltsch_ueber_Troppau</a></td><td>4148</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Meltsch_uber_Troppau\" target=\"_BLANK\">Meltsch_uber_Troppau</a></td><td>4148</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_foreign_La_Liga_players\" target=\"_BLANK\">List_of_foreign_La_Liga_players</a></td><td>4071</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_prehistoric_gastropods\" target=\"_BLANK\">List_of_prehistoric_gastropods</a></td><td>4029</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_marine_gastropod_genera_in_the_fossil_record\" target=\"_BLANK\">List_of_marine_gastropod_genera_in_the_fossil_record</a></td><td>4029</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Loveline_episodes\" target=\"_BLANK\">List_of_Loveline_episodes</a></td><td>4001</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Loveline_Episodes\" target=\"_BLANK\">List_of_Loveline_Episodes</a></td><td>4001</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_urban_areas_in_Sweden\" target=\"_BLANK\">List_of_urban_areas_in_Sweden</a></td><td>3965</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Tamil_films:_1970s\" target=\"_BLANK\">List_of_Tamil_films:_1970s</a></td><td>3893</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Tamil_films_of_the_1970s\" target=\"_BLANK\">Tamil_films_of_the_1970s</a></td><td>3893</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_former_United_States_Senators\" target=\"_BLANK\">List_of_former_United_States_Senators</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_former_members_of_the_U.S._Senate\" target=\"_BLANK\">List_of_former_members_of_the_U.S._Senate</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_former_U.S._Senators\" target=\"_BLANK\">List_of_former_U.S._Senators</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_former_members_of_the_United_States_Senate\" target=\"_BLANK\">List_of_former_members_of_the_United_States_Senate</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_former_US_Senators\" target=\"_BLANK\">List_of_former_US_Senators</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_former_United_States_senators\" target=\"_BLANK\">List_of_former_United_States_senators</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Historic_Members_of_the_United_States_Senate\" target=\"_BLANK\">Historic_Members_of_the_United_States_Senate</a></td><td>3809</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_AM_radio_stations_in_the_United_States_by_call_sign_(initial_letter_K)\" target=\"_BLANK\">List_of_AM_radio_stations_in_the_United_States_by_call_sign_(initial_letter_K)</a></td><td>3803</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_FM_radio_stations_in_the_United_States_by_call_sign_(initial_letters_KN-KZ)\" target=\"_BLANK\">List_of_FM_radio_stations_in_the_United_States_by_call_sign_(initial_letters_KN-KZ)</a></td><td>3792</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Psychology_studies\" target=\"_BLANK\">List_of_Psychology_studies</a></td><td>3784</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_psychological_topics\" target=\"_BLANK\">List_of_psychological_topics</a></td><td>3784</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_psychology_topics\" target=\"_BLANK\">List_of_psychology_topics</a></td><td>3784</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Conservative_Government_1990-1997\" target=\"_BLANK\">Conservative_Government_1990-1997</a></td><td>3754</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Conservative_Government_1979-1997\" target=\"_BLANK\">Conservative_Government_1979-1997</a></td><td>3754</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Conservative_Government_1979-1990\" target=\"_BLANK\">Conservative_Government_1979-1990</a></td><td>3754</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Listed_buildings_in_Liverpool\" target=\"_BLANK\">Listed_buildings_in_Liverpool</a></td><td>3700</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_twin_towns_in_the_Republic_of_Macedonia\" target=\"_BLANK\">List_of_twin_towns_in_the_Republic_of_Macedonia</a></td><td>3698</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_twin_towns_and_sister_cities_in_Poland\" target=\"_BLANK\">List_of_twin_towns_and_sister_cities_in_Poland</a></td><td>3698</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_twin_towns_and_sister_cities_in_Europe\" target=\"_BLANK\">List_of_twin_towns_and_sister_cities_in_Europe</a></td><td>3698</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_twin_towns_and_sister_cities_in_FYROM\" target=\"_BLANK\">List_of_twin_towns_and_sister_cities_in_FYROM</a></td><td>3698</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_cities_and_towns_in_Colorado\" target=\"_BLANK\">List_of_cities_and_towns_in_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Towns_of_Colorado\" target=\"_BLANK\">Towns_of_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Cities_and_towns_of_Colorado\" target=\"_BLANK\">Cities_and_towns_of_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_cities_in_Colorado\" target=\"_BLANK\">List_of_cities_in_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Towns_and_cities_of_Colorado\" target=\"_BLANK\">Towns_and_cities_of_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Towns_and_cities_in_Colorado\" target=\"_BLANK\">Towns_and_cities_in_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Cities_and_towns_in_Colorado\" target=\"_BLANK\">Cities_and_towns_in_Colorado</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Colorado_cities_and_towns\" target=\"_BLANK\">Colorado_cities_and_towns</a></td><td>3653</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sex_topics\" target=\"_BLANK\">Sex_topics</a></td><td>3625</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Unusual_sex_acts\" target=\"_BLANK\">Unusual_sex_acts</a></td><td>3625</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_word_about_sex\" target=\"_BLANK\">List_of_word_about_sex</a></td><td>3625</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_sex-related_topics\" target=\"_BLANK\">List_of_sex-related_topics</a></td><td>3625</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_sexology_topics\" target=\"_BLANK\">List_of_sexology_topics</a></td><td>3625</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_sex_related_topics\" target=\"_BLANK\">List_of_sex_related_topics</a></td><td>3625</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_airports_by_ICAO_code:_K\" target=\"_BLANK\">List_of_airports_by_ICAO_code:_K</a></td><td>3620</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Bosnia_and_Herzegovina-related_topics\" target=\"_BLANK\">List_of_Bosnia_and_Herzegovina-related_topics</a></td><td>3588</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Registered_Historic_Places_in_North_Carolina\" target=\"_BLANK\">List_of_Registered_Historic_Places_in_North_Carolina</a></td><td>3572</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_RHPs_in_NC\" target=\"_BLANK\">List_of_RHPs_in_NC</a></td><td>3572</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/National_Register_of_Historic_Places_listings_in_North_Carolina\" target=\"_BLANK\">National_Register_of_Historic_Places_listings_in_North_Carolina</a></td><td>3572</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_performances_on_Top_of_the_Pops\" target=\"_BLANK\">List_of_performances_on_Top_of_the_Pops</a></td><td>3569</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_performances_on_top_of_the_pops\" target=\"_BLANK\">List_of_performances_on_top_of_the_pops</a></td><td>3569</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Mega_Drive_and_Genesis_games\" target=\"_BLANK\">List_of_Sega_Mega_Drive_and_Genesis_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Genesis_Games\" target=\"_BLANK\">List_of_Sega_Genesis_Games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Sega_Genesis_Game_List\" target=\"_BLANK\">Sega_Genesis_Game_List</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Mega_Drive_games\" target=\"_BLANK\">List_of_Sega_Mega_Drive_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_megadrive_games\" target=\"_BLANK\">List_of_megadrive_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Genesis_games\" target=\"_BLANK\">List_of_Sega_Genesis_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Mega_Drive_Games\" target=\"_BLANK\">List_of_Sega_Mega_Drive_Games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Megadrive_games\" target=\"_BLANK\">List_of_Sega_Megadrive_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Genesis_&_Sega_Mega_Drive_Games\" target=\"_BLANK\">List_of_Sega_Genesis_&_Sega_Mega_Drive_Games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Megadrive_games\" target=\"_BLANK\">List_of_Megadrive_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Mega_Drive/Sega_Genesis_games\" target=\"_BLANK\">List_of_Sega_Mega_Drive/Sega_Genesis_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Sega_Mega_Drive_and_Sega_Genesis_games\" target=\"_BLANK\">List_of_Sega_Mega_Drive_and_Sega_Genesis_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_mega_drive_games\" target=\"_BLANK\">List_of_mega_drive_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Genesis_games\" target=\"_BLANK\">List_of_Genesis_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Mega_Drive_games\" target=\"_BLANK\">List_of_Mega_Drive_games</a></td><td>3568</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_II_topics_(U)\" target=\"_BLANK\">List_of_World_War_II_topics_(U)</a></td><td>3517</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Postal_codes_of_Switzerland\" target=\"_BLANK\">Postal_codes_of_Switzerland</a></td><td>3456</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_postal_codes_of_Switzerland\" target=\"_BLANK\">List_of_postal_codes_of_Switzerland</a></td><td>3456</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Exclusive_xbox_games\" target=\"_BLANK\">Exclusive_xbox_games</a></td><td>3452</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Xbox_games\" target=\"_BLANK\">List_of_Xbox_games</a></td><td>3452</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/XBox_games\" target=\"_BLANK\">XBox_games</a></td><td>3452</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_XBox_games\" target=\"_BLANK\">List_of_XBox_games</a></td><td>3452</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_xbox_games\" target=\"_BLANK\">List_of_xbox_games</a></td><td>3452</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World's_oldest_companies\" target=\"_BLANK\">World's_oldest_companies</a></td><td>3436</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_oldest_family_businesses\" target=\"_BLANK\">List_of_oldest_family_businesses</a></td><td>3436</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Oldest_companies\" target=\"_BLANK\">Oldest_companies</a></td><td>3436</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_oldest_companies\" target=\"_BLANK\">List_of_oldest_companies</a></td><td>3436</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_municipalities_in_Quebec\" target=\"_BLANK\">List_of_municipalities_in_Quebec</a></td><td>3427</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_mathematics_articles_(C)\" target=\"_BLANK\">List_of_mathematics_articles_(C)</a></td><td>3404</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_mathematical_topics_(C)\" target=\"_BLANK\">List_of_mathematical_topics_(C)</a></td><td>3404</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_foreign_Serie_A_players\" target=\"_BLANK\">List_of_foreign_Serie_A_players</a></td><td>3383</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_mathematical_topics_(S)\" target=\"_BLANK\">List_of_mathematical_topics_(S)</a></td><td>3378</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_mathematics_articles_(S)\" target=\"_BLANK\">List_of_mathematics_articles_(S)</a></td><td>3378</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Lord_Commisioner_of_the_Admiralty\" target=\"_BLANK\">Lord_Commisioner_of_the_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Lord_Commissioners_of_the_Admiralty\" target=\"_BLANK\">Lord_Commissioners_of_the_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Lord_of_the_admiralty\" target=\"_BLANK\">Lord_of_the_admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Lord_of_the_Admiralty\" target=\"_BLANK\">Lord_of_the_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Commissioner_of_the_Admiralty\" target=\"_BLANK\">Commissioner_of_the_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Lords_of_the_Admiralty\" target=\"_BLANK\">Lords_of_the_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Lords_Commissioners_of_Admiralty\" target=\"_BLANK\">Lords_Commissioners_of_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Lords_Commissioners_of_the_Admiralty\" target=\"_BLANK\">List_of_Lords_Commissioners_of_the_Admiralty</a></td><td>3374</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_registered_political_parties_in_Spain\" target=\"_BLANK\">List_of_registered_political_parties_in_Spain</a></td><td>3353</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_NASCAR_Drivers\" target=\"_BLANK\">List_of_NASCAR_Drivers</a></td><td>3319</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/NASCAR/FamousDrivers\" target=\"_BLANK\">NASCAR/FamousDrivers</a></td><td>3319</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_NASCAR_drivers\" target=\"_BLANK\">List_of_NASCAR_drivers</a></td><td>3319</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_famous_NASCAR_drivers\" target=\"_BLANK\">List_of_famous_NASCAR_drivers</a></td><td>3319</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Warships_of_World_War_Two\" target=\"_BLANK\">Warships_of_World_War_Two</a></td><td>3294</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Warships_of_world_war_II\" target=\"_BLANK\">Warships_of_world_war_II</a></td><td>3294</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ships_of_World_War_II\" target=\"_BLANK\">List_of_ships_of_World_War_II</a></td><td>3294</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_War_II_ships\" target=\"_BLANK\">List_of_World_War_II_ships</a></td><td>3294</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Naval_Vessels_of_World_War_II\" target=\"_BLANK\">List_of_Naval_Vessels_of_World_War_II</a></td><td>3294</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Malaysia-related_topics\" target=\"_BLANK\">List_of_Malaysia-related_topics</a></td><td>3271</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_guns\" target=\"_BLANK\">List_of_guns</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Iceland_topics\" target=\"_BLANK\">Iceland_topics</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_iceland_related_topics\" target=\"_BLANK\">List_of_iceland_related_topics</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Iceland-related_articles\" target=\"_BLANK\">List_of_Iceland-related_articles</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/ASAI_One-Pro\" target=\"_BLANK\">ASAI_One-Pro</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Icelandic_topics\" target=\"_BLANK\">List_of_Icelandic_topics</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_firearms\" target=\"_BLANK\">List_of_firearms</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_iceland_topics\" target=\"_BLANK\">List_of_iceland_topics</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Iceland-related_topics\" target=\"_BLANK\">List_of_Iceland-related_topics</a></td><td>3251</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_FIFA_World_Cup-_Trinidad_and_Tobago\" target=\"_BLANK\">2006_FIFA_World_Cup-_Trinidad_and_Tobago</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_Football_World_Cup_(squads)\" target=\"_BLANK\">2006_Football_World_Cup_(squads)</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_FIFA_World_Cup_(squads)\" target=\"_BLANK\">2006_FIFA_World_Cup_(squads)</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Germany_2006_rosters\" target=\"_BLANK\">Germany_2006_rosters</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_FIFA_World_Cup_-_Netherlands\" target=\"_BLANK\">2006_FIFA_World_Cup_-_Netherlands</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_FIFA_World_Cup_-_Spain\" target=\"_BLANK\">2006_FIFA_World_Cup_-_Spain</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Germany_2006_squads\" target=\"_BLANK\">Germany_2006_squads</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World_cup_squads\" target=\"_BLANK\">World_cup_squads</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Football_World_Cup_2006_(squads)\" target=\"_BLANK\">Football_World_Cup_2006_(squads)</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World_cup_2006_squads\" target=\"_BLANK\">World_cup_2006_squads</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World_cup_2006_roster\" target=\"_BLANK\">World_cup_2006_roster</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_FIFA_World_Cup_squads\" target=\"_BLANK\">2006_FIFA_World_Cup_squads</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_world_cup_squads\" target=\"_BLANK\">2006_world_cup_squads</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/England_world_cup_squad\" target=\"_BLANK\">England_world_cup_squad</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/World_cup_2006_rosters\" target=\"_BLANK\">World_cup_2006_rosters</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/FIFA_World_Cup_2006_(squads)\" target=\"_BLANK\">FIFA_World_Cup_2006_(squads)</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_fifa_world_cup_rosters\" target=\"_BLANK\">2006_fifa_world_cup_rosters</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_FIFA_World_Cup_-_Trinidad_and_Tobago\" target=\"_BLANK\">2006_FIFA_World_Cup_-_Trinidad_and_Tobago</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Fifa_world_cup_2006_squads\" target=\"_BLANK\">Fifa_world_cup_2006_squads</a></td><td>3240</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Flora_of_Ohio\" target=\"_BLANK\">Flora_of_Ohio</a></td><td>3238</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Iowa_townships\" target=\"_BLANK\">List_of_Iowa_townships</a></td><td>3214</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_townships_in_Iowa\" target=\"_BLANK\">List_of_townships_in_Iowa</a></td><td>3214</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Chile-related_topics\" target=\"_BLANK\">List_of_Chile-related_topics</a></td><td>3207</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Singapore-related_topics_by_alphabetical_order\" target=\"_BLANK\">List_of_Singapore-related_topics_by_alphabetical_order</a></td><td>3206</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Asian_birds\" target=\"_BLANK\">List_of_Asian_birds</a></td><td>3199</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/National_Register_of_Historic_Places_listings_in_Arkansas\" target=\"_BLANK\">National_Register_of_Historic_Places_listings_in_Arkansas</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Georgia_(U.S._state):_S-Z\" target=\"_BLANK\">List_of_places_in_Georgia_(U.S._state):_S-Z</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Registered_Historic_Places_listings_in_Arkansas\" target=\"_BLANK\">List_of_Registered_Historic_Places_listings_in_Arkansas</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_RHPs_in_AR\" target=\"_BLANK\">List_of_RHPs_in_AR</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Georgia,_USA:_S-Z\" target=\"_BLANK\">List_of_places_in_Georgia,_USA:_S-Z</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_Georgia_U.S._state):_S-Z\" target=\"_BLANK\">List_of_places_in_Georgia_U.S._state):_S-Z</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Registered_Historic_Places_in_Arkansas\" target=\"_BLANK\">List_of_Registered_Historic_Places_in_Arkansas</a></td><td>3194</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_ATP_Challenger_Series\" target=\"_BLANK\">2008_ATP_Challenger_Series</a></td><td>3185</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_German_scheduled_railway_routes\" target=\"_BLANK\">List_of_German_scheduled_railway_routes</a></td><td>3173</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Kursbuchstrecke\" target=\"_BLANK\">Kursbuchstrecke</a></td><td>3173</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_scheduled_railway_routes_in_Germany\" target=\"_BLANK\">List_of_scheduled_railway_routes_in_Germany</a></td><td>3173</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Church_of_the_Annunciation_(Nazareth)\" target=\"_BLANK\">Church_of_the_Annunciation_(Nazareth)</a></td><td>3164</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_basilicas\" target=\"_BLANK\">List_of_basilicas</a></td><td>3164</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/United_States_Congressional_Delegations_from_Pennsylvania\" target=\"_BLANK\">United_States_Congressional_Delegations_from_Pennsylvania</a></td><td>3153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/United_States_Congressional_Delegations_from_PA\" target=\"_BLANK\">United_States_Congressional_Delegations_from_PA</a></td><td>3153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/US_Congressional_Delegations_from_Pennsylvania\" target=\"_BLANK\">US_Congressional_Delegations_from_Pennsylvania</a></td><td>3153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/U.S._Congressional_Delegations_from_Pennsylvania\" target=\"_BLANK\">U.S._Congressional_Delegations_from_Pennsylvania</a></td><td>3153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/United_States_congressional_delegations_from_Pennsylvania\" target=\"_BLANK\">United_States_congressional_delegations_from_Pennsylvania</a></td><td>3153</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Nannites\" target=\"_BLANK\">Nannites</a></td><td>3137</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ammonites\" target=\"_BLANK\">List_of_ammonites</a></td><td>3137</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_English_Test_cricketers\" target=\"_BLANK\">List_of_English_Test_cricketers</a></td><td>3131</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_England_Test_cricketers\" target=\"_BLANK\">List_of_England_Test_cricketers</a></td><td>3131</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Protected_areas_of_the_People's_Republic_of_China\" target=\"_BLANK\">Protected_areas_of_the_People's_Republic_of_China</a></td><td>3129</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Protected_Areas_of_the_People's_Republic_of_China\" target=\"_BLANK\">Protected_Areas_of_the_People's_Republic_of_China</a></td><td>3129</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Bamboo_forest\" target=\"_BLANK\">Bamboo_forest</a></td><td>3129</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Protected_areas_in_China\" target=\"_BLANK\">Protected_areas_in_China</a></td><td>3129</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/NGC_1-999\" target=\"_BLANK\">NGC_1-999</a></td><td>3119</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_NGC_objects_(1-999)\" target=\"_BLANK\">List_of_NGC_objects_(1-999)</a></td><td>3119</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_municipalities_in_Germany\" target=\"_BLANK\">List_of_municipalities_in_Germany</a></td><td>3119</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Ty_Beanie_Babies\" target=\"_BLANK\">List_of_Ty_Beanie_Babies</a></td><td>3116</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Bijoux_(Beanie_Baby)\" target=\"_BLANK\">Bijoux_(Beanie_Baby)</a></td><td>3116</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_preserved_steam_locomotives_in_Germany\" target=\"_BLANK\">List_of_preserved_steam_locomotives_in_Germany</a></td><td>3111</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_U.S._counties_in_alphabetical_order\" target=\"_BLANK\">List_of_U.S._counties_in_alphabetical_order</a></td><td>3100</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_US_counties_in_alphabetical_order\" target=\"_BLANK\">List_of_US_counties_in_alphabetical_order</a></td><td>3100</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_rodents\" target=\"_BLANK\">List_of_rodents</a></td><td>3096</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_placental_mammals_in_Order_Rodentia\" target=\"_BLANK\">List_of_placental_mammals_in_Order_Rodentia</a></td><td>3096</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_fashion_topics\" target=\"_BLANK\">List_of_fashion_topics</a></td><td>3094</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_the_Netherlands\" target=\"_BLANK\">List_of_places_in_the_Netherlands</a></td><td>3083</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Dutch_places\" target=\"_BLANK\">List_of_Dutch_places</a></td><td>3083</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_operas_by_composer\" target=\"_BLANK\">List_of_operas_by_composer</a></td><td>3073</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/The_Opera_Corpus\" target=\"_BLANK\">The_Opera_Corpus</a></td><td>3073</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Famous_operettas\" target=\"_BLANK\">Famous_operettas</a></td><td>3073</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_operettas\" target=\"_BLANK\">List_of_operettas</a></td><td>3073</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/The_opera_corpus\" target=\"_BLANK\">The_opera_corpus</a></td><td>3073</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Opera_corpus\" target=\"_BLANK\">Opera_corpus</a></td><td>3073</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_county_subdivisions_in_Michigan\" target=\"_BLANK\">List_of_county_subdivisions_in_Michigan</a></td><td>3072</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_museums_in_New_York\" target=\"_BLANK\">List_of_museums_in_New_York</a></td><td>3067</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Museums_in_San_Diego\" target=\"_BLANK\">List_of_Museums_in_San_Diego</a></td><td>3062</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Museums_in_San_Francisco\" target=\"_BLANK\">List_of_Museums_in_San_Francisco</a></td><td>3062</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_museums_in_California\" target=\"_BLANK\">List_of_museums_in_California</a></td><td>3062</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_museums_in_San_Francisco\" target=\"_BLANK\">List_of_museums_in_San_Francisco</a></td><td>3062</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_museums_in_San_Diego\" target=\"_BLANK\">List_of_museums_in_San_Diego</a></td><td>3062</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Olympic_champions/Track_and_Field_-_Men\" target=\"_BLANK\">Olympic_champions/Track_and_Field_-_Men</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_10000_metres\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_10000_metres</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Olympic_athletics_medalists_(men)\" target=\"_BLANK\">Olympic_athletics_medalists_(men)</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_1500_metres\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_1500_metres</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Olympic_medalists_in_athletics_(men)\" target=\"_BLANK\">List_of_Olympic_medalists_in_athletics_(men)</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_110_metre_hurdles\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_110_metre_hurdles</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_400_metre_hurdles\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_400_metre_hurdles</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Olympic_medalists_in_athletics_(men)\" target=\"_BLANK\">Olympic_medalists_in_athletics_(men)</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_4x100_metre_relay\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_4x100_metre_relay</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_400_metres\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_400_metres</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_800_metres\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_800_metres</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_4x400_metre_relay\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_4x400_metre_relay</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_places_in_the_UTC-5_timezone\" target=\"_BLANK\">List_of_places_in_the_UTC-5_timezone</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_50km_walk\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_50km_walk</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_decathlon\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_decathlon</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_200_metres\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_200_metres</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Athletics_at_the_1932_Summer_Olympics_-_Men's_100_metres\" target=\"_BLANK\">Athletics_at_the_1932_Summer_Olympics_-_Men's_100_metres</a></td><td>3060</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_songs_by_The_Beach_Boys\" target=\"_BLANK\">List_of_songs_by_The_Beach_Boys</a></td><td>3059</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_The_Beach_Boys_songs\" target=\"_BLANK\">List_of_The_Beach_Boys_songs</a></td><td>3059</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Beach_Boys_complete_song_list\" target=\"_BLANK\">Beach_Boys_complete_song_list</a></td><td>3059</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Political_families_by_country:_A-E\" target=\"_BLANK\">Political_families_by_country:_A-E</a></td><td>3056</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Political_families_by_country:_N-Z\" target=\"_BLANK\">Political_families_by_country:_N-Z</a></td><td>3056</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Political_Families\" target=\"_BLANK\">List_of_Political_Families</a></td><td>3056</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_political_families\" target=\"_BLANK\">List_of_political_families</a></td><td>3056</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Political_families_of_the_world\" target=\"_BLANK\">Political_families_of_the_world</a></td><td>3056</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Political_families_by_country:_F-M\" target=\"_BLANK\">Political_families_by_country:_F-M</a></td><td>3056</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Tamil_films:_1960s\" target=\"_BLANK\">List_of_Tamil_films:_1960s</a></td><td>3046</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Tamil_films_of_the_1960s\" target=\"_BLANK\">Tamil_films_of_the_1960s</a></td><td>3046</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Padma_Bhushan\" target=\"_BLANK\">Padma_Bhushan</a></td><td>3034</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Padmabhushan\" target=\"_BLANK\">Padmabhushan</a></td><td>3034</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Statewide_opinion_polling_for_the_United_States_presidential_election,_2008\" target=\"_BLANK\">Statewide_opinion_polling_for_the_United_States_presidential_election,_2008</a></td><td>3024</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Statewide_Opinion_Polling_for_the_United_States_Presidential_Election,_2008\" target=\"_BLANK\">Statewide_Opinion_Polling_for_the_United_States_Presidential_Election,_2008</a></td><td>3024</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Opinion_polling_by_state_for_the_United_States_presidential_election,_2008\" target=\"_BLANK\">Opinion_polling_by_state_for_the_United_States_presidential_election,_2008</a></td><td>3024</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Statewide_opinion_polling_for_the_us_presidential_election_2008\" target=\"_BLANK\">Statewide_opinion_polling_for_the_us_presidential_election_2008</a></td><td>3024</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Dreamcast_Games\" target=\"_BLANK\">List_of_Dreamcast_Games</a></td><td>3019</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Dreamcast_games\" target=\"_BLANK\">List_of_Dreamcast_games</a></td><td>3019</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_dreamcast_games\" target=\"_BLANK\">List_of_dreamcast_games</a></td><td>3019</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Dreamcast_games\" target=\"_BLANK\">Dreamcast_games</a></td><td>3019</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Birds_of_Australasia\" target=\"_BLANK\">Birds_of_Australasia</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Australian,_New_Zealand_and_Antarctic_birds\" target=\"_BLANK\">List_of_Australian,_New_Zealand_and_Antarctic_birds</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/HANZAB_list\" target=\"_BLANK\">HANZAB_list</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_birds_of_Australasia\" target=\"_BLANK\">List_of_birds_of_Australasia</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_birds_of_Australia,_New_Zealand_and_Antarctica\" target=\"_BLANK\">List_of_birds_of_Australia,_New_Zealand_and_Antarctica</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Australasian_birds\" target=\"_BLANK\">Australasian_birds</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Australasian_birds\" target=\"_BLANK\">List_of_Australasian_birds</a></td><td>3008</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_museum_ships\" target=\"_BLANK\">List_of_museum_ships</a></td><td>2996</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_American_Civil_War_Medal_of_Honor_recipients:_M-Z\" target=\"_BLANK\">List_of_American_Civil_War_Medal_of_Honor_recipients:_M-Z</a></td><td>2996</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Medal_of_Honor_recipients_for_the_American_Civil_War:_M-Z\" target=\"_BLANK\">List_of_Medal_of_Honor_recipients_for_the_American_Civil_War:_M-Z</a></td><td>2996</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Medal_of_Honor_recipients:_Civil_War_M-Z\" target=\"_BLANK\">List_of_Medal_of_Honor_recipients:_Civil_War_M-Z</a></td><td>2996</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_American_Civil_War_Medal_of_Honor_recipients:_M\u2013Z\" target=\"_BLANK\">List_of_American_Civil_War_Medal_of_Honor_recipients:_M\u2013Z</a></td><td>2996</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_railway_stations_in_Japan:_K-L\" target=\"_BLANK\">List_of_railway_stations_in_Japan:_K-L</a></td><td>2993</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Railway_Stations_in_Japan/KL\" target=\"_BLANK\">List_of_Railway_Stations_in_Japan/KL</a></td><td>2993</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_United_States_Representatives_from_California\" target=\"_BLANK\">List_of_United_States_Representatives_from_California</a></td><td>2989</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2007_(Expected_Events)\" target=\"_BLANK\">2007_(Expected_Events)</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Two_Thousand_and_Seven\" target=\"_BLANK\">Two_Thousand_and_Seven</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2007\" target=\"_BLANK\">2007</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2007_AD\" target=\"_BLANK\">2007_AD</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2007_calendar\" target=\"_BLANK\">2007_calendar</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Novemeber_2007\" target=\"_BLANK\">Novemeber_2007</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Year_2007\" target=\"_BLANK\">Year_2007</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/MMVII\" target=\"_BLANK\">MMVII</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Two_thousand_and_Seven\" target=\"_BLANK\">Two_thousand_and_Seven</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2K7\" target=\"_BLANK\">2K7</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2007_events\" target=\"_BLANK\">2007_events</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Heisei_19\" target=\"_BLANK\">Heisei_19</a></td><td>2973</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Yanqiao_Fang\" target=\"_BLANK\">Yanqiao_Fang</a></td><td>2957</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/China_at_the_2008_Summer_Olympics\" target=\"_BLANK\">China_at_the_2008_Summer_Olympics</a></td><td>2957</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Fang_Yanqiao\" target=\"_BLANK\">Fang_Yanqiao</a></td><td>2957</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/China_at_the_2008_Olympics\" target=\"_BLANK\">China_at_the_2008_Olympics</a></td><td>2957</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Shang_Ruihua\" target=\"_BLANK\">Shang_Ruihua</a></td><td>2957</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_F1_drivers\" target=\"_BLANK\">List_of_F1_drivers</a></td><td>2943</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Formula_One_drivers\" target=\"_BLANK\">List_of_Formula_One_drivers</a></td><td>2943</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/January_2009_in_sports\" target=\"_BLANK\">January_2009_in_sports</a></td><td>2939</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Pennsylvania_locations_by_per_capita_income\" target=\"_BLANK\">List_of_Pennsylvania_locations_by_per_capita_income</a></td><td>2939</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Pennsylvania_locations_by_per_capita_income\" target=\"_BLANK\">Pennsylvania_locations_by_per_capita_income</a></td><td>2939</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Richest_Places_in_Pennsylvania\" target=\"_BLANK\">Richest_Places_in_Pennsylvania</a></td><td>2939</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_in_Music\" target=\"_BLANK\">2006_in_Music</a></td><td>2929</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_music\" target=\"_BLANK\">2006_music</a></td><td>2929</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2006_in_music\" target=\"_BLANK\">2006_in_music</a></td><td>2929</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Music_in_2006\" target=\"_BLANK\">Music_in_2006</a></td><td>2929</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Coalition_casualties_in_Afghanistan\" target=\"_BLANK\">Coalition_casualties_in_Afghanistan</a></td><td>2925</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Coalition_Casualties_in_Afghanistan\" target=\"_BLANK\">Coalition_Casualties_in_Afghanistan</a></td><td>2925</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_University_of_Pennsylvania_people\" target=\"_BLANK\">List_of_University_of_Pennsylvania_people</a></td><td>2919</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Frank_Zappa_musicians\" target=\"_BLANK\">List_of_Frank_Zappa_musicians</a></td><td>2904</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/1998_FIFA_World_Cup_(squads)\" target=\"_BLANK\">1998_FIFA_World_Cup_(squads)</a></td><td>2898</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/1998_FIFA_World_Cup_squads\" target=\"_BLANK\">1998_FIFA_World_Cup_squads</a></td><td>2898</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Football_World_Cup_1998_(squads)\" target=\"_BLANK\">Football_World_Cup_1998_(squads)</a></td><td>2898</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/1998_world_cup_squads\" target=\"_BLANK\">1998_world_cup_squads</a></td><td>2898</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Australian_plant_species_authored_by_Ferdinand_von_Mueller\" target=\"_BLANK\">List_of_Australian_plant_species_authored_by_Ferdinand_von_Mueller</a></td><td>2893</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_United_States_Representatives_from_New_York\" target=\"_BLANK\">List_of_United_States_Representatives_from_New_York</a></td><td>2881</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Ohio_townships\" target=\"_BLANK\">Ohio_townships</a></td><td>2858</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_townships_in_Ohio\" target=\"_BLANK\">List_of_townships_in_Ohio</a></td><td>2858</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Common_township_names_in_Ohio\" target=\"_BLANK\">Common_township_names_in_Ohio</a></td><td>2858</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Ohio_townships\" target=\"_BLANK\">List_of_Ohio_townships</a></td><td>2858</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_historic_buildings_and_architects_of_the_United_Kingdom\" target=\"_BLANK\">List_of_historic_buildings_and_architects_of_the_United_Kingdom</a></td><td>2855</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Historic_buildings_and_architects_of_the_United_Kingdom\" target=\"_BLANK\">Historic_buildings_and_architects_of_the_United_Kingdom</a></td><td>2855</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_service_craft_of_the_United_States_Navy\" target=\"_BLANK\">List_of_service_craft_of_the_United_States_Navy</a></td><td>2845</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_auxiliaries_of_the_United_States_Navy\" target=\"_BLANK\">List_of_auxiliaries_of_the_United_States_Navy</a></td><td>2845</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/US_Navy_auxiliary_ships\" target=\"_BLANK\">US_Navy_auxiliary_ships</a></td><td>2845</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/AGOR\" target=\"_BLANK\">AGOR</a></td><td>2845</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_twin_towns_and_sister_cities_in_France\" target=\"_BLANK\">List_of_twin_towns_and_sister_cities_in_France</a></td><td>2843</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_2008_films\" target=\"_BLANK\">List_of_2008_films</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_World_Championships_medalists_in_sailing_(centreboard_classes)\" target=\"_BLANK\">List_of_World_Championships_medalists_in_sailing_(centreboard_classes)</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_2008_movies\" target=\"_BLANK\">List_of_2008_movies</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_movie\" target=\"_BLANK\">2008_movie</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_in_movies\" target=\"_BLANK\">2008_in_movies</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Movies_in_2008\" target=\"_BLANK\">Movies_in_2008</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_movies\" target=\"_BLANK\">2008_movies</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_film\" target=\"_BLANK\">2008_film</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_films\" target=\"_BLANK\">2008_films</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/2008_in_film\" target=\"_BLANK\">2008_in_film</a></td><td>2819</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_guitarists_(Blues,_Jazz,_Rock,_Metal)\" target=\"_BLANK\">List_of_guitarists_(Blues,_Jazz,_Rock,_Metal)</a></td><td>2817</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_guitarists\" target=\"_BLANK\">List_of_guitarists</a></td><td>2817</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_Of_Guitar_Players\" target=\"_BLANK\">List_Of_Guitar_Players</a></td><td>2817</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Ontario_communities\" target=\"_BLANK\">List_of_Ontario_communities</a></td><td>2806</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_communities_in_Ontario\" target=\"_BLANK\">List_of_communities_in_Ontario</a></td><td>2806</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Communities_of_the_Province_of_Ontario,_Canada\" target=\"_BLANK\">Communities_of_the_Province_of_Ontario,_Canada</a></td><td>2806</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_television_stations_in_Mexico_by_call_sign\" target=\"_BLANK\">List_of_television_stations_in_Mexico_by_call_sign</a></td><td>2797</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_television_stations_in_Mexico\" target=\"_BLANK\">List_of_television_stations_in_Mexico</a></td><td>2797</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_1-9\" target=\"_BLANK\">List_of_Turkish_films_by_name:_1-9</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_B\" target=\"_BLANK\">List_of_Turkish_films_by_name:_B</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_G\" target=\"_BLANK\">List_of_Turkish_films_by_name:_G</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_M\" target=\"_BLANK\">List_of_Turkish_films_by_name:_M</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_A\" target=\"_BLANK\">List_of_Turkish_films_by_name:_A</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films:_A-Z\" target=\"_BLANK\">List_of_Turkish_films:_A-Z</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_N\" target=\"_BLANK\">List_of_Turkish_films_by_name:_N</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_E\" target=\"_BLANK\">List_of_Turkish_films_by_name:_E</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_Z\" target=\"_BLANK\">List_of_Turkish_films_by_name:_Z</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_Y\" target=\"_BLANK\">List_of_Turkish_films_by_name:_Y</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_R\" target=\"_BLANK\">List_of_Turkish_films_by_name:_R</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_S\" target=\"_BLANK\">List_of_Turkish_films_by_name:_S</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_P\" target=\"_BLANK\">List_of_Turkish_films_by_name:_P</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_V\" target=\"_BLANK\">List_of_Turkish_films_by_name:_V</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_T\" target=\"_BLANK\">List_of_Turkish_films_by_name:_T</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_U\" target=\"_BLANK\">List_of_Turkish_films_by_name:_U</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_J\" target=\"_BLANK\">List_of_Turkish_films_by_name:_J</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_K\" target=\"_BLANK\">List_of_Turkish_films_by_name:_K</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_H\" target=\"_BLANK\">List_of_Turkish_films_by_name:_H</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_I\" target=\"_BLANK\">List_of_Turkish_films_by_name:_I</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_O\" target=\"_BLANK\">List_of_Turkish_films_by_name:_O</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_L\" target=\"_BLANK\">List_of_Turkish_films_by_name:_L</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_C\" target=\"_BLANK\">List_of_Turkish_films_by_name:_C</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_F\" target=\"_BLANK\">List_of_Turkish_films_by_name:_F</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Turkish_films_by_name:_D\" target=\"_BLANK\">List_of_Turkish_films_by_name:_D</a></td><td>2794</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/December_2008_in_sports\" target=\"_BLANK\">December_2008_in_sports</a></td><td>2793</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Members_of_the_110th_United_States_Congress\" target=\"_BLANK\">Members_of_the_110th_United_States_Congress</a></td><td>2785</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_townships_in_Missouri\" target=\"_BLANK\">List_of_townships_in_Missouri</a></td><td>2765</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_zx_spectrum_games\" target=\"_BLANK\">List_of_zx_spectrum_games</a></td><td>2762</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ZX_Spectrum_games\" target=\"_BLANK\">List_of_ZX_Spectrum_games</a></td><td>2762</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_spectrum_games\" target=\"_BLANK\">List_of_spectrum_games</a></td><td>2762</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_schools_in_Norway\" target=\"_BLANK\">List_of_schools_in_Norway</a></td><td>2760</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_ecoregions_(WWF)\" target=\"_BLANK\">List_of_ecoregions_(WWF)</a></td><td>2753</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_films:_S\" target=\"_BLANK\">List_of_films:_S</a></td><td>2750</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_philosophical_topics_(I-Q)\" target=\"_BLANK\">List_of_philosophical_topics_(I-Q)</a></td><td>2749</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_philosophy_topics_(I-Q)\" target=\"_BLANK\">List_of_philosophy_topics_(I-Q)</a></td><td>2749</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Acts_of_Parliament_of_the_United_Kingdom_Parliament,_1780-1800\" target=\"_BLANK\">List_of_Acts_of_Parliament_of_the_United_Kingdom_Parliament,_1780-1800</a></td><td>2743</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Acts_of_Parliament_of_the_Great_Britain_Parliament,_1780-1800\" target=\"_BLANK\">List_of_Acts_of_Parliament_of_the_Great_Britain_Parliament,_1780-1800</a></td><td>2743</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Acts_of_Parliament_of_the_United_Kingdom_Parliament,_1780-1799\" target=\"_BLANK\">List_of_Acts_of_Parliament_of_the_United_Kingdom_Parliament,_1780-1799</a></td><td>2743</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_Special_elections_to_the_United_States_House_of_Representatives\" target=\"_BLANK\">List_of_Special_elections_to_the_United_States_House_of_Representatives</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Special_elections_to_the_United_States_House_of_Representatives\" target=\"_BLANK\">Special_elections_to_the_United_States_House_of_Representatives</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Summary_of_special_elections_to_the_United_States_House_of_Representatives_by_Congress\" target=\"_BLANK\">Summary_of_special_elections_to_the_United_States_House_of_Representatives_by_Congress</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_special_elections_to_the_United_States_House_of_Representatives\" target=\"_BLANK\">List_of_special_elections_to_the_United_States_House_of_Representatives</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_United_States_House_of_Representatives_special_elections\" target=\"_BLANK\">List_of_United_States_House_of_Representatives_special_elections</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Special_Elections_to_the_United_States_House_of_Representatives\" target=\"_BLANK\">Special_Elections_to_the_United_States_House_of_Representatives</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_special_elections_to_the_United_States_House_of_Representatives_(1st_to_26th_Congresses;_1789-1841)\" target=\"_BLANK\">List_of_special_elections_to_the_United_States_House_of_Representatives_(1st_to_26th_Congresses;_1789-1841)</a></td><td>2741</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Droga_wojewodzka\" target=\"_BLANK\">Droga_wojewodzka</a></td><td>2738</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Voivodship_Road\" target=\"_BLANK\">Voivodship_Road</a></td><td>2738</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Droga_wojew\u00f3dzka\" target=\"_BLANK\">Droga_wojew\u00f3dzka</a></td><td>2738</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Voivodship_road\" target=\"_BLANK\">Voivodship_road</a></td><td>2738</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/Voivodeship_road\" target=\"_BLANK\">Voivodeship_road</a></td><td>2738</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_mythical_creatures\" target=\"_BLANK\">List_of_mythical_creatures</a></td><td>2734</td></tr><tr><td><a href=\"http://en.wikipedia.org/wiki/List_of_species_in_folklore_and_mythology\" target=\"_BLANK\">List_of_species_in_folklore_and_mythology</a></td><td>2734</td></tr></table>"
],
"metadata": {},
"output_type": "pyout",
"prompt_number": 41,
"text": [
"<IPython.core.display.HTML at 0x391cb50>"
]
}
],
"prompt_number": 41
}
],
"metadata": {}
}
]
}
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment