Skip to content

Commit beac6c8

Browse files
fix: unify logging, address Pylint warnings
1 parent 02be757 commit beac6c8

File tree

2 files changed

+40
-28
lines changed

2 files changed

+40
-28
lines changed

indexwarcs_cc_pyspark.py

Lines changed: 19 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,10 @@
1-
import logging
2-
from sparkcc import CCFileProcessorSparkJob
3-
1+
from gzip import GzipFile
42
from tempfile import TemporaryFile
3+
54
from pywb.indexer.cdxindexer import write_cdx_index
6-
from gzip import GzipFile
75

6+
from sparkcc import CCFileProcessorSparkJob
87

9-
LOG = logging.getLogger('IndexWARCJob')
108

119

1210
class IndexWARCJob(CCFileProcessorSparkJob):
@@ -18,6 +16,15 @@ class IndexWARCJob(CCFileProcessorSparkJob):
1816

1917
name = 'IndexWARCJob'
2018

19+
# description of input and output shown by --help
20+
input_descr = "Path to file listing input paths (WARC/WAT/WET/ARC)"
21+
output_descr = """Table containing the output CDX files
22+
(in spark.sql.warehouse.dir) and the indexing status:
23+
1 successfully created,
24+
0 already exists,
25+
-1 processing failed"""
26+
27+
# PyWB index options
2128
index_options = {
2229
'surt_ordered': True,
2330
'sort': True,
@@ -29,8 +36,9 @@ def add_arguments(self, parser):
2936
super(CCFileProcessorSparkJob, self).add_arguments(parser)
3037
parser.add_argument("--output_base_url", required=True,
3138
help="Destination for CDX output.")
32-
parser.add_argument("--skip-existing", dest='skip_existing', action='store_true',
33-
help="Skip processing files for which the output CDX file already exists.")
39+
parser.add_argument("--skip_existing", action='store_true',
40+
help="Skip processing files for which "
41+
"the output CDX file already exists.")
3442

3543
def _conv_warc_to_cdx_path(self, warc_path):
3644
cdx_path = warc_path.replace('crawl-data', 'cc-index/cdx')
@@ -43,11 +51,11 @@ def process_file(self, warc_path, tempfd):
4351

4452
cdx_path = self._conv_warc_to_cdx_path(warc_path)
4553

46-
LOG.info('Indexing WARC: %s', warc_path)
54+
self.get_logger().info('Indexing WARC: %s', warc_path)
4755

4856
if self.args.skip_existing and \
4957
self.check_for_output_file(cdx_path,self.args.output_base_url):
50-
LOG.info('Already Exists: %s', cdx_path)
58+
self.get_logger().info('Already Exists: %s', cdx_path)
5159
yield cdx_path, 0
5260
return
5361

@@ -60,14 +68,14 @@ def process_file(self, warc_path, tempfd):
6068
write_cdx_index(cdxfile, tempfd, warc_path, **self.index_options)
6169
success = True
6270
except Exception as exc:
63-
LOG.error('Failed to index %s: %s', warc_path, exc)
71+
self.get_logger().error('Failed to index %s: %s', warc_path, exc)
6472

6573
cdxtemp.flush()
6674
cdxtemp.seek(0)
6775

6876
if success:
6977
self.write_output_file(cdx_path, cdxtemp, self.args.output_base_url)
70-
LOG.info('Successfully uploaded CDX: %s', cdx_path)
78+
self.get_logger().info('Successfully uploaded CDX: %s', cdx_path)
7179
yield cdx_path, 1
7280
else:
7381
yield cdx_path, -1

zipnumcluster_cc_pyspark.py

Lines changed: 21 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,6 @@ class ZipNumClusterCdx(CCFileProcessorSparkJob):
2424

2525
name = 'ZipNumClusterCdx'
2626

27-
LOG = logging.getLogger('ZipNumClusterCdx')
28-
2927
DATA_URL_PATTERN = re.compile('^(s3|https?|file|hdfs|s3a|s3n):(?://([^/]*))?/(.*)')
3028

3129

@@ -34,8 +32,9 @@ def add_arguments(self, parser):
3432
parser.add_argument("--output_base_url", required=True,
3533
help="Output destination.")
3634
parser.add_argument("--partition_boundaries_file", required=True,
37-
help="Full path to a JSON file containing partition boundaries."
38-
"If specified, and does not exist, will be created, otherwise, will be used.")
35+
help="Full path to a JSON file containing partition boundaries. "
36+
"If specified, and does not exist, will be created, otherwise, "
37+
"it will be used.")
3938
parser.add_argument("--temporary_output_base_url", required=True,
4039
help="Temporary output location for per-shard cluster indexes.")
4140
parser.add_argument("--num_lines", type=int, required=False,
@@ -93,25 +92,27 @@ def write_output_file(uri, fd, base_uri=None):
9392
else:
9493
# keep local file paths as is
9594
path = uri
95+
scheme = 'file'
96+
netloc = None
9697

97-
if scheme in ['s3', 's3a', 's3n']:
98+
if scheme in {'s3', 's3a', 's3n'}:
9899
bucketname = netloc
99100
output_path = path
100101
try:
101102
client = boto3.client('s3')
102103
client.upload_fileobj(fd, bucketname, path)
103104
except botocore.client.ClientError as exception:
104-
ZipNumClusterCdx.LOG.error(
105+
logging.error(
105106
'Failed to write to S3 {}: {}'.format(output_path, exception))
106107

107-
elif scheme == 'http' or scheme == 'https':
108+
elif scheme in {'http', 'https'}:
108109
raise ValueError('HTTP/HTTPS output not supported')
109110

110111
elif scheme == 'hdfs':
111112
raise NotImplementedError('HDFS output not implemented')
112113

113114
else:
114-
ZipNumClusterCdx.LOG.info('Writing local file {}'.format(uri))
115+
logging.info('Writing local file {}'.format(uri))
115116
if scheme == 'file':
116117
# must be an absolute path
117118
uri = os.path.join('/', path)
@@ -130,7 +131,7 @@ def write_partition_with_global_seq(idx: int, partition_iter: list,
130131
# Calculate starting sequence number for this partition
131132
start_seq = (idx * records_per_partition) + 1 if records_per_partition else 1
132133

133-
with open(partition_idx_file, 'w') as f:
134+
with open(partition_idx_file, 'w', encoding="utf-8") as f:
134135
seq = start_seq
135136
for record in partition_iter:
136137
min_surt, _, min_surt_timestamp, filename, _, offset, length, _ = record
@@ -157,6 +158,7 @@ def process_partition(partition_id: int, partition_iter: Iterator[Tuple[str, Tup
157158
current_chunk = []
158159
chunk_min_surt = None
159160
chunk_max_surt = None
161+
chunk_min_timestamp = None
160162

161163
with open(output_filename, 'wb') as f:
162164
for (surt_key, timestamp), json_data in partition_iter:
@@ -222,25 +224,26 @@ def process_partition(partition_id: int, partition_iter: Iterator[Tuple[str, Tup
222224
return final_files
223225

224226
def run_job(self, session):
225-
input = self.args.input_base_url + self.args.input
227+
input_url = self.args.input_base_url + self.args.input
226228
num_partitions = self.args.num_output_partitions
227229
boundaries_file_uri = self.args.partition_boundaries_file
228230
num_lines = self.args.num_lines
229231
output_base_url = self.args.output_base_url
230232
temporary_output_base_url = self.args.temporary_output_base_url
231233

232-
rdd = session.sparkContext.textFile(input).map(
234+
rdd = session.sparkContext.textFile(input_url).map(
233235
self.parse_line).filter(lambda x: x is not None)
234236

235237
boundaries = None
236-
logging.info(f"Boundaries file: {boundaries_file_uri}")
238+
self.get_logger(session).info(f"Boundaries file: {boundaries_file_uri}")
237239
if boundaries_file_uri and self.check_for_output_file(boundaries_file_uri):
238-
logging.info(f"Boundaries file found, using it: {boundaries_file_uri}")
240+
self.get_logger(session).info(f"Boundaries file found, using it: {boundaries_file_uri}")
239241
with self.fetch_file(boundaries_file_uri) as f:
240242
boundaries = list(map(lambda l: tuple(l), json.load(f)))
241243

242244
else:
243-
# this percent needs to be pretty small, since this collect brings data back to driver...
245+
# The percentage needs to be pretty small, since the collect
246+
# brings data back to the driver...
244247
# 1/2 percent should be fine
245248
samples = rdd.keys().sample(False, 0.005).collect()
246249
samples.sort()
@@ -254,15 +257,16 @@ def run_job(self, session):
254257
boundaries.append(samples[idx])
255258

256259
temp_file_name = 'temp_range_boundaries.json'
257-
with open(temp_file_name, 'w') as f:
260+
with open(temp_file_name, 'w', encoding="utf-8") as f:
258261
json.dump(boundaries, f)
259262

260263
with open(temp_file_name, 'rb') as f:
261264
self.write_output_file(boundaries_file_uri, f)
262265

263266
os.unlink(temp_file_name)
264267

265-
logging.info(f"Boundaries file created: {boundaries_file_uri}")
268+
self.get_logger(session).info(
269+
f"Boundaries file created: {boundaries_file_uri}")
266270

267271
rdd = rdd.repartitionAndSortWithinPartitions(
268272
numPartitions=num_partitions,
@@ -287,4 +291,4 @@ def run_job(self, session):
287291

288292
if __name__ == "__main__":
289293
job = ZipNumClusterCdx()
290-
job.run()
294+
job.run()

0 commit comments

Comments
 (0)