warcprox/warcprox/dedup.py

570 lines
24 KiB
Python
Raw Normal View History

2017-03-02 15:06:26 -08:00
'''
warcprox/dedup.py - identical payload digest deduplication using sqlite db
2017-03-02 15:06:26 -08:00
2018-01-16 11:37:56 -08:00
Copyright (C) 2013-2018 Internet Archive
2017-03-02 15:06:26 -08:00
This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License
as published by the Free Software Foundation; either version 2
of the License, or (at your option) any later version.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU General Public License for more details.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301,
USA.
'''
2016-04-06 19:37:55 -07:00
2015-03-18 16:29:44 -07:00
from __future__ import absolute_import
2014-11-15 03:20:05 -08:00
import logging
import os
import json
from hanzo import warctools
import warcprox
import warcprox.trough
import sqlite3
import doublethink
import datetime
import urllib3
from urllib3.exceptions import HTTPError
2018-01-16 11:37:56 -08:00
import collections
from concurrent import futures
urllib3.disable_warnings()
2014-11-15 03:20:05 -08:00
class DedupableMixin(object):
def __init__(self, options=warcprox.Options()):
self.min_text_size = options.dedup_min_text_size or 0
self.min_binary_size = options.dedup_min_binary_size or 0
self.dedup_only_with_bucket = options.dedup_only_with_bucket or False
def should_dedup(self, recorded_url):
"""Check if we should try to run dedup on resource based on payload
size compared with min text/binary dedup size options.
When we use option --dedup-only-with-bucket, `dedup-bucket` is required
in Warcprox-Meta to perform dedup.
Return Boolean.
"""
if self.dedup_only_with_bucket and "dedup-bucket" not in recorded_url.warcprox_meta:
return False
if recorded_url.is_text():
return recorded_url.response_recorder.payload_size() > self.min_text_size
else:
return recorded_url.response_recorder.payload_size() > self.min_binary_size
class DedupLoader(warcprox.BaseStandardPostfetchProcessor, DedupableMixin):
def __init__(self, dedup_db, options=warcprox.Options()):
warcprox.BaseStandardPostfetchProcessor.__init__(self, options=options)
DedupableMixin.__init__(self, options)
self.dedup_db = dedup_db
def _process_url(self, recorded_url):
if (recorded_url.response_recorder
and recorded_url.payload_digest
and self.should_dedup(recorded_url)):
digest_key = warcprox.digest_str(recorded_url.payload_digest, self.options.base32)
if recorded_url.warcprox_meta and "dedup-bucket" in recorded_url.warcprox_meta:
recorded_url.dedup_info = self.dedup_db.lookup(
digest_key, recorded_url.warcprox_meta["dedup-bucket"],
recorded_url.url)
else:
recorded_url.dedup_info = self.dedup_db.lookup(
digest_key, url=recorded_url.url)
class DedupDb(DedupableMixin):
2015-03-18 16:29:44 -07:00
logger = logging.getLogger("warcprox.dedup.DedupDb")
2014-11-15 03:20:05 -08:00
def __init__(
self, file='./warcprox.sqlite', options=warcprox.Options()):
DedupableMixin.__init__(self, options)
self.file = file
self.options = options
2014-11-15 03:20:05 -08:00
def start(self):
if os.path.exists(self.file):
self.logger.info(
'opening existing deduplication database %s',
self.file)
else:
self.logger.info(
'creating new deduplication database %s', self.file)
conn = sqlite3.connect(self.file)
conn.execute(
'create table if not exists dedup ('
' key varchar(300) primary key,'
' value varchar(4000)'
');')
conn.commit()
conn.close()
def loader(self, *args, **kwargs):
return DedupLoader(self, self.options)
def storer(self, *args, **kwargs):
return warcprox.ListenerPostfetchProcessor(self, self.options)
def save(self, digest_key, response_record, bucket=""):
2014-11-15 03:20:05 -08:00
record_id = response_record.get_header(warctools.WarcRecord.ID).decode('latin1')
url = response_record.get_header(warctools.WarcRecord.URL).decode('latin1')
date = response_record.get_header(warctools.WarcRecord.DATE).decode('latin1')
key = digest_key.decode('utf-8') + "|" + bucket
2015-08-20 21:46:40 +00:00
py_value = {'id':record_id, 'url':url, 'date':date}
2014-11-15 03:20:05 -08:00
json_value = json.dumps(py_value, separators=(',',':'))
conn = sqlite3.connect(self.file)
conn.execute(
'insert or replace into dedup (key, value) values (?, ?)',
(key, json_value))
conn.commit()
conn.close()
self.logger.debug('dedup db saved %s:%s', key, json_value)
2014-11-15 03:20:05 -08:00
def lookup(self, digest_key, bucket="", url=None):
result = None
key = digest_key.decode('utf-8') + '|' + bucket
conn = sqlite3.connect(self.file)
cursor = conn.execute('select value from dedup where key = ?', (key,))
result_tuple = cursor.fetchone()
conn.close()
if result_tuple:
result = json.loads(result_tuple[0])
2015-08-20 21:46:40 +00:00
result['id'] = result['id'].encode('latin1')
result['url'] = result['url'].encode('latin1')
result['date'] = result['date'].encode('latin1')
self.logger.debug('dedup db lookup of key=%s returning %s', key, result)
return result
2014-11-15 03:20:05 -08:00
def notify(self, recorded_url, records):
if (records and records[0].type == b'response'
and self.should_dedup(recorded_url)):
digest_key = warcprox.digest_str(
recorded_url.payload_digest, self.options.base32)
if recorded_url.warcprox_meta and "dedup-bucket" in recorded_url.warcprox_meta:
self.save(
digest_key, records[0],
bucket=recorded_url.warcprox_meta["dedup-bucket"])
else:
self.save(digest_key, records[0])
class RethinkDedupDb(DedupDb, DedupableMixin):
2015-08-20 21:46:40 +00:00
logger = logging.getLogger("warcprox.dedup.RethinkDedupDb")
def __init__(self, options=warcprox.Options()):
DedupableMixin.__init__(self, options)
parsed = doublethink.parse_rethinkdb_url(options.rethinkdb_dedup_url)
self.rr = doublethink.Rethinker(
servers=parsed.hosts, db=parsed.database)
self.table = parsed.table
2015-08-20 21:46:40 +00:00
self._ensure_db_table()
self.options = options
2015-08-20 21:46:40 +00:00
def _ensure_db_table(self):
2017-03-02 15:06:26 -08:00
dbs = self.rr.db_list().run()
if not self.rr.dbname in dbs:
2017-06-07 16:05:47 -07:00
self.logger.info("creating rethinkdb database %r", self.rr.dbname)
2017-03-02 15:06:26 -08:00
self.rr.db_create(self.rr.dbname).run()
tables = self.rr.table_list().run()
if not self.table in tables:
2017-03-02 15:06:26 -08:00
self.logger.info(
2017-06-07 16:05:47 -07:00
"creating rethinkdb table %r in database %r shards=%r "
"replicas=%r", self.table, self.rr.dbname,
len(self.rr.servers), min(3, len(self.rr.servers)))
2017-03-02 15:06:26 -08:00
self.rr.table_create(
self.table, primary_key="key", shards=len(self.rr.servers),
replicas=min(3, len(self.rr.servers))).run()
def start(self):
pass
def save(self, digest_key, response_record, bucket=""):
k = digest_key.decode("utf-8") if isinstance(digest_key, bytes) else digest_key
k = "{}|{}".format(k, bucket)
2015-08-20 21:46:40 +00:00
record_id = response_record.get_header(warctools.WarcRecord.ID).decode('latin1')
url = response_record.get_header(warctools.WarcRecord.URL).decode('latin1')
date = response_record.get_header(warctools.WarcRecord.DATE).decode('latin1')
record = {'key':k,'url':url,'date':date,'id':record_id}
2017-03-02 15:06:26 -08:00
result = self.rr.table(self.table).insert(
record, conflict="replace").run()
if sorted(result.values()) != [0,0,0,0,0,1] and [result["deleted"],result["skipped"],result["errors"]] != [0,0,0]:
raise Exception("unexpected result %s saving %s", result, record)
self.logger.debug('dedup db saved %s:%s', k, record)
2015-08-20 21:46:40 +00:00
def lookup(self, digest_key, bucket="", url=None):
k = digest_key.decode("utf-8") if isinstance(digest_key, bytes) else digest_key
k = "{}|{}".format(k, bucket)
2017-03-02 15:06:26 -08:00
result = self.rr.table(self.table).get(k).run()
if result:
for x in result:
result[x] = result[x].encode("utf-8")
self.logger.debug('dedup db lookup of key=%s returning %s', k, result)
return result
def notify(self, recorded_url, records):
if (records and records[0].type == b'response'
and self.should_dedup(recorded_url)):
digest_key = warcprox.digest_str(
recorded_url.payload_digest, self.options.base32)
if recorded_url.warcprox_meta and "dedup-bucket" in recorded_url.warcprox_meta:
self.save(digest_key, records[0], bucket=recorded_url.warcprox_meta["dedup-bucket"])
else:
self.save(digest_key, records[0])
2017-03-02 15:06:26 -08:00
class CdxServerDedup(DedupDb):
"""Query a CDX server to perform deduplication.
"""
logger = logging.getLogger("warcprox.dedup.CdxServerDedup")
cookies = None
2017-10-19 22:54:34 +00:00
def __init__(self, cdx_url="https://web.archive.org/cdx/search",
maxsize=400, options=warcprox.Options()):
"""Initialize cdx server connection pool and related parameters.
Use low timeout value and no retries to avoid blocking warcprox
operation by a slow CDX server.
"""
self.cdx_url = cdx_url
self.options = options
2018-04-04 06:15:52 +00:00
headers = {'User-Agent': 'warcprox', 'Accept-Encoding': 'gzip, deflate'}
if options.cdxserver_dedup_cookies:
headers['Cookie'] = options.cdxserver_dedup_cookies
self.http_pool = urllib3.PoolManager(maxsize=maxsize, retries=0,
timeout=2.0, headers=headers)
2018-01-23 23:16:35 +00:00
def loader(self, *args, **kwargs):
return CdxServerDedupLoader(self, self.options)
def start(self):
pass
def save(self, digest_key, response_record, bucket=""):
"""Does not apply to CDX server, as it is obviously read-only.
"""
pass
def lookup(self, digest_key, url):
"""Compare `sha1` with SHA1 hash of fetched content (note SHA1 must be
computed on the original content, after decoding Content-Encoding and
Transfer-Encoding, if any), if they match, write a revisit record.
Get only the last item (limit=-1) because Wayback Machine has special
performance optimisation to handle that. limit < 0 is very inefficient
in general. Maybe it could be configurable in the future.
:param digest_key: b'sha1:<KEY-VALUE>' (prefix is optional).
Example: b'sha1:B2LTWWPUOYAH7UIPQ7ZUPQ4VMBSVC36A'
:param url: Target URL string
Result must contain:
{"url": <URL>, "date": "%Y-%m-%dT%H:%M:%SZ"}
"""
u = url.decode("utf-8") if isinstance(url, bytes) else url
try:
result = self.http_pool.request('GET', self.cdx_url, fields=dict(
url=u, fl="timestamp,digest", filter="!mimetype:warc/revisit",
limit=-1))
assert result.status == 200
if isinstance(digest_key, bytes):
dkey = digest_key
else:
dkey = digest_key.encode('utf-8')
dkey = dkey[5:] if dkey.startswith(b'sha1:') else dkey
line = result.data.strip()
if line:
(cdx_ts, cdx_digest) = line.split(b' ')
if cdx_digest == dkey:
Merge branch 'master' into trough-dedup * master: Update docstring Move Warcprox-Meta header construction to warcproxy Improve test_writer tests Replace timestamp parameter with more generic request/response syntax Return capture timestamp Swap fcntl.flock with fcntl.lockf Unit test fix for Python2 compatibility Test WarcWriter file locking when no_warc_open_suffix=True Rename writer var and add exception handling Acquire and exclusive file lock when not using .open WARC suffix Add hidden --no-warc-open-suffix CLI option Fix missing dummy url param in bigtable lookup method back to dev version number version 2.2 for pypi to address https://github.com/internetarchive/warcprox/issues/42 Expand comment with limit=-1 explanation Drop unnecessary split for newline in CDX results fix benchmarks (update command line args) Update CdxServerDedup lookup algorithm Pass url instead of recorded_url obj to dedup lookup methods Filter out warc/revisit records in CdxServerDedup Improve CdxServerDedup implementation Fix minor CdxServerDedup unit test Fix bug with dedup_info date encoding Add mock pkg to run-tests.sh Add CdxServerDedup unit tests and improve its exception handling Add CDX Server based deduplication cryptography lib version 2.1.1 is causing problems Revert changes to test_warcprox.py Revert changes to bigtable and dedup Revert warc to previous behavior Update unit test Replace invalid warcfilename variable in playback Stop using WarcRecord.REFERS_TO header and use payload_digest instead
2017-11-02 16:34:52 -07:00
dt = datetime.datetime.strptime(
cdx_ts.decode('ascii'), '%Y%m%d%H%M%S')
date = dt.strftime('%Y-%m-%dT%H:%M:%SZ').encode('utf-8')
return dict(url=url, date=date)
except (HTTPError, AssertionError, ValueError) as exc:
self.logger.error('CdxServerDedup request failed for url=%s %s',
url, exc)
return None
def notify(self, recorded_url, records):
"""Since we don't save anything to CDX server, this does not apply.
"""
pass
Merge branch 'master' into trough-dedup * master: Update docstring Move Warcprox-Meta header construction to warcproxy Improve test_writer tests Replace timestamp parameter with more generic request/response syntax Return capture timestamp Swap fcntl.flock with fcntl.lockf Unit test fix for Python2 compatibility Test WarcWriter file locking when no_warc_open_suffix=True Rename writer var and add exception handling Acquire and exclusive file lock when not using .open WARC suffix Add hidden --no-warc-open-suffix CLI option Fix missing dummy url param in bigtable lookup method back to dev version number version 2.2 for pypi to address https://github.com/internetarchive/warcprox/issues/42 Expand comment with limit=-1 explanation Drop unnecessary split for newline in CDX results fix benchmarks (update command line args) Update CdxServerDedup lookup algorithm Pass url instead of recorded_url obj to dedup lookup methods Filter out warc/revisit records in CdxServerDedup Improve CdxServerDedup implementation Fix minor CdxServerDedup unit test Fix bug with dedup_info date encoding Add mock pkg to run-tests.sh Add CdxServerDedup unit tests and improve its exception handling Add CDX Server based deduplication cryptography lib version 2.1.1 is causing problems Revert changes to test_warcprox.py Revert changes to bigtable and dedup Revert warc to previous behavior Update unit test Replace invalid warcfilename variable in playback Stop using WarcRecord.REFERS_TO header and use payload_digest instead
2017-11-02 16:34:52 -07:00
class CdxServerDedupLoader(warcprox.BaseBatchPostfetchProcessor, DedupableMixin):
2018-01-23 23:16:35 +00:00
def __init__(self, cdx_dedup, options=warcprox.Options()):
warcprox.BaseBatchPostfetchProcessor.__init__(self, options)
DedupableMixin.__init__(self, options)
self.pool = futures.ThreadPoolExecutor(max_workers=400)
2018-01-23 23:16:35 +00:00
self.batch = set()
self.cdx_dedup = cdx_dedup
def _get_process_put(self):
recorded_url = self.inq.get(block=True, timeout=0.5)
if (recorded_url.response_recorder
and recorded_url.payload_digest
and self.should_dedup(recorded_url)):
self.batch.add(recorded_url)
self.pool.submit(self._process_url, recorded_url)
else:
if self.outq:
self.outq.put(recorded_url)
2018-01-23 23:16:35 +00:00
def _process_url(self, recorded_url):
try:
digest_key = warcprox.digest_str(recorded_url.payload_digest,
self.options.base32)
dedup_info = self.cdx_dedup.lookup(digest_key, recorded_url.url)
if dedup_info:
recorded_url.dedup_info = dedup_info
except ValueError as exc:
self.logger.error('CdxServerDedupLoader _process_url failed for url=%s %s',
recorded_url.url, exc)
finally:
self.batch.remove(recorded_url)
if self.outq:
self.outq.put(recorded_url)
class BatchTroughStorer(warcprox.BaseBatchPostfetchProcessor):
2018-01-17 16:49:28 -08:00
def __init__(self, trough_dedup_db, options=warcprox.Options()):
warcprox.BaseBatchPostfetchProcessor.__init__(self, options)
self.trough_dedup_db = trough_dedup_db
def _filter_and_bucketize(self, batch):
'''
2018-01-24 16:07:45 -08:00
Returns `{bucket: [recorded_url, ...]}`, excluding urls that should not
2018-01-17 16:49:28 -08:00
have dedup info stored.
'''
buckets = collections.defaultdict(list)
for recorded_url in batch:
if (recorded_url.warc_records
and recorded_url.warc_records[0].type == b'response'
and self.trough_dedup_db.should_dedup(recorded_url)):
2018-01-17 16:49:28 -08:00
if (recorded_url.warcprox_meta
and 'dedup-bucket' in recorded_url.warcprox_meta):
bucket = recorded_url.warcprox_meta['dedup-bucket']
2018-01-17 16:49:28 -08:00
else:
bucket = '__unspecified__'
buckets[bucket].append(recorded_url)
return buckets
def _process_batch(self, batch):
buckets = self._filter_and_bucketize(batch)
if not buckets:
return
fs = {}
with futures.ThreadPoolExecutor(max_workers=len(buckets)) as pool:
# send off requests in parallel
for bucket in buckets:
future = pool.submit(
self.trough_dedup_db.batch_save,
buckets[bucket], bucket)
fs[future] = bucket
# wait for results
try:
for future in futures.as_completed(fs, timeout=20):
pass
except futures.TimeoutError as e:
# the remaining threads actually keep running in this case,
# there's no way to stop them, but that should be harmless
logging.warn(
'timed out saving dedup info to trough', exc_info=True)
2018-01-17 16:49:28 -08:00
class BatchTroughLoader(warcprox.BaseBatchPostfetchProcessor):
logger = logging.getLogger("warcprox.dedup.BatchTroughLoader")
def __init__(self, trough_dedup_db, options=warcprox.Options()):
warcprox.BaseBatchPostfetchProcessor.__init__(self, options)
2018-01-16 11:37:56 -08:00
self.trough_dedup_db = trough_dedup_db
2018-01-17 16:49:28 -08:00
def _startup(self):
self.trough_dedup_db.start()
2018-01-16 11:37:56 -08:00
def _filter_and_bucketize(self, batch):
'''
Returns `{bucket: [recorded_url, ...]}`, excluding urls that should not
be looked up.
'''
buckets = collections.defaultdict(list)
discards = []
2018-01-16 11:37:56 -08:00
for recorded_url in batch:
if (recorded_url.response_recorder
and recorded_url.payload_digest
and self.trough_dedup_db.should_dedup(recorded_url)):
2018-01-17 16:49:28 -08:00
if (recorded_url.warcprox_meta
and 'dedup-bucket' in recorded_url.warcprox_meta):
bucket = recorded_url.warcprox_meta['dedup-bucket']
2018-01-16 11:37:56 -08:00
else:
bucket = '__unspecified__'
buckets[bucket].append(recorded_url)
else:
discards.append(
warcprox.digest_str(
recorded_url.payload_digest, self.options.base32)
if recorded_url.payload_digest else 'n/a')
self.logger.debug(
'filtered out digests (not loading dedup): %r', discards)
2018-01-16 11:37:56 -08:00
return buckets
def _build_key_index(self, batch):
'''
Builds index of RecordedUrl by digest key.
Args:
batch(list): list of RecordedUrl
Returns:
dict `{digest_key: [recorded_url, ...]}`
2018-01-16 11:37:56 -08:00
'''
key_index = collections.defaultdict(list)
for recorded_url in batch:
digest_key = warcprox.digest_str(
recorded_url.payload_digest, self.options.base32)
key_index[digest_key].append(recorded_url)
return key_index
def _process_batch(self, batch):
buckets = self._filter_and_bucketize(batch)
if not buckets:
return
fs = {}
with futures.ThreadPoolExecutor(max_workers=len(buckets)) as pool:
# send off the trough requests in parallel
for bucket in buckets:
key_index = self._build_key_index(buckets[bucket])
future = pool.submit(
self.trough_dedup_db.batch_lookup,
key_index.keys(), bucket)
fs[future] = bucket
# process results as they come back
try:
for future in futures.as_completed(fs, timeout=20):
bucket = fs[future]
try:
for entry in future.result():
for recorded_url in key_index[entry['digest_key']]:
recorded_url.dedup_info = entry
except Exception as e:
# batch_lookup raised exception or something
logging.warn(
'problem looking up dedup info for %s urls '
'in bucket %s', len(buckets[bucket]), bucket,
exc_info=True)
if self.logger.isEnabledFor(logging.DEBUG):
dups = sorted([e['digest_key'] for e in future.result()])
novel = sorted([
k for k in key_index.keys() if k not in dups])
self.logger.debug(
'bucket %s: dups=%r novel=%r',
bucket, dups, novel)
except futures.TimeoutError as e:
# the remaining threads actually keep running in this case,
# there's no way to stop them, but that should be harmless
self.logger.warn(
'timed out loading dedup info from trough', exc_info=True)
2018-01-16 11:37:56 -08:00
class TroughDedupDb(DedupDb, DedupableMixin):
'''
https://github.com/internetarchive/trough
'''
logger = logging.getLogger("warcprox.dedup.TroughDedupDb")
SCHEMA_ID = 'warcprox-dedup-v1'
SCHEMA_SQL = ('create table dedup (\n'
' digest_key varchar(100) primary key,\n'
' url varchar(2100) not null,\n'
' date datetime not null,\n'
' id varchar(100));\n') # warc record id
2018-01-17 16:49:28 -08:00
WRITE_SQL_TMPL = ('insert or ignore into dedup\n'
'(digest_key, url, date, id)\n'
'values (%s, %s, %s, %s);')
def __init__(self, options=warcprox.Options()):
DedupableMixin.__init__(self, options)
self.options = options
self._trough_cli = warcprox.trough.TroughClient(
2017-11-13 14:22:17 -08:00
options.rethinkdb_trough_db_url, promotion_interval=60*60)
2018-01-17 16:49:28 -08:00
def loader(self, *args, **kwargs):
return BatchTroughLoader(self, self.options)
def storer(self, *args, **kwargs):
return BatchTroughStorer(self, self.options)
2018-01-16 11:37:56 -08:00
def start(self):
self._trough_cli.register_schema(self.SCHEMA_ID, self.SCHEMA_SQL)
def save(self, digest_key, response_record, bucket='__unspecified__'):
record_id = response_record.get_header(warctools.WarcRecord.ID)
url = response_record.get_header(warctools.WarcRecord.URL)
warc_date = response_record.get_header(warctools.WarcRecord.DATE)
self._trough_cli.write(
bucket, self.WRITE_SQL_TMPL,
(digest_key, url, warc_date, record_id), self.SCHEMA_ID)
2018-01-17 16:49:28 -08:00
def batch_save(self, batch, bucket='__unspecified__'):
sql_tmpl = ('insert or ignore into dedup\n'
'(digest_key, url, date, id)\n'
'values %s;' % ','.join(
'(%s,%s,%s,%s)' for i in range(len(batch))))
values = []
for recorded_url in batch:
values.extend([
warcprox.digest_str(
recorded_url.payload_digest, self.options.base32),
recorded_url.url,
recorded_url.warc_records[0].date,
recorded_url.warc_records[0].id,])
self._trough_cli.write(bucket, sql_tmpl, values, self.SCHEMA_ID)
Merge branch 'master' into trough-dedup * master: Update docstring Move Warcprox-Meta header construction to warcproxy Improve test_writer tests Replace timestamp parameter with more generic request/response syntax Return capture timestamp Swap fcntl.flock with fcntl.lockf Unit test fix for Python2 compatibility Test WarcWriter file locking when no_warc_open_suffix=True Rename writer var and add exception handling Acquire and exclusive file lock when not using .open WARC suffix Add hidden --no-warc-open-suffix CLI option Fix missing dummy url param in bigtable lookup method back to dev version number version 2.2 for pypi to address https://github.com/internetarchive/warcprox/issues/42 Expand comment with limit=-1 explanation Drop unnecessary split for newline in CDX results fix benchmarks (update command line args) Update CdxServerDedup lookup algorithm Pass url instead of recorded_url obj to dedup lookup methods Filter out warc/revisit records in CdxServerDedup Improve CdxServerDedup implementation Fix minor CdxServerDedup unit test Fix bug with dedup_info date encoding Add mock pkg to run-tests.sh Add CdxServerDedup unit tests and improve its exception handling Add CDX Server based deduplication cryptography lib version 2.1.1 is causing problems Revert changes to test_warcprox.py Revert changes to bigtable and dedup Revert warc to previous behavior Update unit test Replace invalid warcfilename variable in playback Stop using WarcRecord.REFERS_TO header and use payload_digest instead
2017-11-02 16:34:52 -07:00
def lookup(self, digest_key, bucket='__unspecified__', url=None):
results = self._trough_cli.read(
bucket, 'select * from dedup where digest_key=%s;',
(digest_key,))
if results:
assert len(results) == 1 # sanity check (digest_key is primary key)
result = results[0]
result['id'] = result['id'].encode('ascii')
result['url'] = result['url'].encode('ascii')
result['date'] = result['date'].encode('ascii')
self.logger.debug(
'trough lookup of key=%r returning %r', digest_key, result)
return result
else:
return None
2018-01-16 11:37:56 -08:00
def batch_lookup(self, digest_keys, bucket='__unspecified__'):
'''Returns [{'digest_key': ..., 'url': ..., 'date': ...}, ...]'''
2018-01-16 11:37:56 -08:00
sql_tmpl = 'select * from dedup where digest_key in (%s)' % (
','.join('%s' for i in range(len(digest_keys))))
results = self._trough_cli.read(bucket, sql_tmpl, digest_keys)
if results is None:
return []
self.logger.debug(
'trough batch lookup of %s keys returned %s results',
len(digest_keys), len(results))
assert len(results) >= 0 and len(results) <= len(digest_keys)
for result in results:
result['id'] = result.get('id') and result['id'].encode('ascii')
2018-01-16 11:37:56 -08:00
result['url'] = result['url'].encode('ascii')
result['date'] = result['date'].encode('ascii')
result['digest_key'] = result['digest_key'].encode('ascii')
return results
def notify(self, recorded_url, records):
if (records and records[0].type == b'response'
and self.should_dedup(recorded_url)):
digest_key = warcprox.digest_str(
recorded_url.payload_digest, self.options.base32)
if recorded_url.warcprox_meta and 'dedup-bucket' in recorded_url.warcprox_meta:
self.save(
digest_key, records[0],
bucket=recorded_url.warcprox_meta['dedup-bucket'])
else:
self.save(digest_key, records[0])