github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/dataframe/doctests.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 17 """A module that allows running existing pandas doctests with Beam dataframes. 18 19 This module hooks into the doctesting framework by providing a custom 20 runner and, in particular, an OutputChecker, as well as providing a fake 21 object for mocking out the pandas module. 22 23 The (novel) sequence of events when running a doctest is as follows. 24 25 1. The test invokes `pd.DataFrame(...)` (or similar) and an actual dataframe 26 is computed and stashed but a Beam deferred dataframe is returned 27 in its place. 28 2. Computations are done on these "dataframes," resulting in new objects, 29 but as these are actually deferred, only expression trees are built. 30 In the background, a mapping of id -> deferred dataframe is stored for 31 each newly created dataframe. 32 3. When any dataframe is printed out, the repr has been overwritten to 33 print `Dataframe[id]`. The aforementened mapping is used to map this back 34 to the actual dataframe object, which is then computed via Beam, and its 35 the (stringified) result plugged into the actual output for comparison. 36 4. The comparison is then done on the sorted lines of the expected and actual 37 values. 38 """ 39 40 import collections 41 import contextlib 42 import doctest 43 import re 44 import sys 45 import traceback 46 from io import StringIO 47 from typing import Any 48 from typing import Dict 49 from typing import List 50 51 import numpy as np 52 import pandas as pd 53 54 import apache_beam as beam 55 from apache_beam.dataframe import expressions 56 from apache_beam.dataframe import frames # pylint: disable=unused-import 57 from apache_beam.dataframe import pandas_top_level_functions 58 from apache_beam.dataframe import transforms 59 from apache_beam.dataframe.frame_base import DeferredBase 60 61 62 class FakePandasObject(object): 63 """A stand-in for the wrapped pandas objects. 64 """ 65 def __init__(self, pandas_obj, test_env): 66 self._pandas_obj = pandas_obj 67 self._test_env = test_env 68 69 def __call__(self, *args, **kwargs): 70 result = self._pandas_obj(*args, **kwargs) 71 if type(result) in DeferredBase._pandas_type_map: 72 placeholder = expressions.PlaceholderExpression(result.iloc[0:0]) 73 self._test_env._inputs[placeholder] = result 74 return DeferredBase.wrap(placeholder) 75 else: 76 return result 77 78 def __getattr__(self, name): 79 attr = getattr(self._pandas_obj, name) 80 if callable(attr): 81 result = FakePandasObject(attr, self._test_env) 82 else: 83 result = attr 84 # Cache this so two lookups return the same object. 85 setattr(self, name, result) 86 return result 87 88 def __reduce__(self): 89 return lambda: pd, () 90 91 92 class TestEnvironment(object): 93 """A class managing the patching (of methods, inputs, and outputs) needed 94 to run and validate tests. 95 96 These classes are patched to be able to recognize and retrieve inputs 97 and results, stored in `self._inputs` and `self._all_frames` respectively. 98 """ 99 def __init__(self): 100 self._inputs = {} 101 self._all_frames = {} 102 103 def fake_pandas_module(self): 104 return FakePandasObject(pandas_top_level_functions.pd_wrapper, self) 105 106 @contextlib.contextmanager 107 def _monkey_patch_type(self, deferred_type): 108 """Monkey-patch __init__ to record a pointer to all created frames, and 109 __repr__ to be able to recognize them in the doctest output. 110 """ 111 try: 112 old_init, old_repr = deferred_type.__init__, deferred_type.__repr__ 113 114 def new_init(df, *args, **kwargs): 115 old_init(df, *args, **kwargs) 116 self._all_frames[id(df)] = df 117 118 deferred_type.__init__ = new_init 119 deferred_type.__repr__ = lambda self: 'DeferredBase[%s]' % id(self) 120 self._recorded_results = collections.defaultdict(list) 121 yield 122 finally: 123 deferred_type.__init__, deferred_type.__repr__ = old_init, old_repr 124 125 @contextlib.contextmanager 126 def context(self): 127 """Creates a context within which DeferredBase types are monkey patched 128 to record ids.""" 129 with contextlib.ExitStack() as stack: 130 for deferred_type in DeferredBase._pandas_type_map.values(): 131 stack.enter_context(self._monkey_patch_type(deferred_type)) 132 yield 133 134 135 class _InMemoryResultRecorder(object): 136 """Helper for extracting computed results from a Beam pipeline. 137 138 Used as follows:: 139 140 with _InMemoryResultRecorder() as recorder: 141 with beam.Pipeline() as p: 142 ... 143 pcoll | beam.Map(recorder.record_fn(name)) 144 145 seen = recorder.get_recorded(name) 146 """ 147 148 # Class-level value to survive pickling. 149 _ALL_RESULTS = {} # type: Dict[str, List[Any]] 150 151 def __init__(self): 152 self._id = id(self) 153 154 def __enter__(self): 155 self._ALL_RESULTS[self._id] = collections.defaultdict(list) 156 return self 157 158 def __exit__(self, *unused_args): 159 del self._ALL_RESULTS[self._id] 160 161 def record_fn(self, name): 162 def record(value): 163 self._ALL_RESULTS[self._id][name].append(value) 164 165 return record 166 167 def get_recorded(self, name): 168 return self._ALL_RESULTS[self._id][name] 169 170 171 WONT_IMPLEMENT = 'apache_beam.dataframe.frame_base.WontImplementError' 172 NOT_IMPLEMENTED = 'NotImplementedError' 173 174 175 class _DeferrredDataframeOutputChecker(doctest.OutputChecker): 176 """Validates output by replacing DeferredBase[...] with computed values. 177 """ 178 def __init__(self, env, use_beam): 179 self._env = env 180 if use_beam: 181 self.compute = self.compute_using_beam 182 else: 183 self.compute = self.compute_using_session 184 self.reset() 185 186 def reset(self): 187 self._last_error = None 188 189 def compute_using_session(self, to_compute): 190 session = expressions.PartitioningSession(self._env._inputs) 191 return { 192 name: session.evaluate(frame._expr) 193 for name, 194 frame in to_compute.items() 195 } 196 197 def compute_using_beam(self, to_compute): 198 with _InMemoryResultRecorder() as recorder: 199 with beam.Pipeline() as p: 200 input_pcolls = { 201 placeholder: p 202 | 'Create%s' % placeholder >> beam.Create([input[::2], input[1::2]]) 203 for placeholder, 204 input in self._env._inputs.items() 205 } 206 output_pcolls = ( 207 input_pcolls | transforms._DataframeExpressionsTransform( 208 {name: frame._expr 209 for name, frame in to_compute.items()})) 210 for name, output_pcoll in output_pcolls.items(): 211 _ = output_pcoll | 'Record%s' % name >> beam.FlatMap( 212 recorder.record_fn(name)) 213 # pipeline runs, side effects recorded 214 215 def concat(values): 216 if len(values) > 1: 217 return pd.concat(values) 218 else: 219 return values[0] 220 221 return { 222 name: concat(recorder.get_recorded(name)) 223 for name in to_compute.keys() 224 } 225 226 def fix(self, want, got): 227 if 'DeferredBase' in got: 228 try: 229 to_compute = { 230 m.group(0): self._env._all_frames[int(m.group(1))] 231 for m in re.finditer(r'DeferredBase\[(\d+)\]', got) 232 } 233 computed = self.compute(to_compute) 234 for name, frame in computed.items(): 235 got = got.replace(name, repr(frame)) 236 237 # If a multiindex is used, compensate for it 238 if any(isinstance(frame, pd.core.generic.NDFrame) and 239 frame.index.nlevels > 1 for frame in computed.values()): 240 241 def fill_multiindex(text): 242 """An awful hack to work around the fact that pandas omits repeated 243 elements in a multi-index. 244 For example: 245 246 Series name Row ID 247 s1 0 a 248 1 b 249 s2 0 c 250 1 d 251 dtype: object 252 253 The s1 and s2 are implied for the 2nd and 4th rows. However if we 254 re-order this Series it might be printed this way: 255 256 Series name Row ID 257 s1 0 a 258 s2 1 d 259 s2 0 c 260 s1 1 b 261 dtype: object 262 263 In our model these are equivalent, but when we sort the lines and 264 check equality they are not. This method fills in any omitted 265 multiindex values, so that we can successfully sort and compare.""" 266 lines = [list(line) for line in text.split('\n')] 267 for prev, line in zip(lines[:-1], lines[1:]): 268 if all(l == ' ' for l in line): 269 continue 270 271 for i, l in enumerate(line): 272 if l != ' ': 273 break 274 line[i] = prev[i] 275 276 return '\n'.join(''.join(line) for line in lines) 277 278 got = fill_multiindex(got) 279 want = fill_multiindex(want) 280 281 def sort_and_normalize(text): 282 return '\n'.join( 283 sorted( 284 [line.rstrip() for line in text.split('\n') if line.strip()], 285 key=str.strip)) + '\n' 286 287 got = sort_and_normalize(got) 288 want = sort_and_normalize(want) 289 except Exception: 290 got = traceback.format_exc() 291 return want, got 292 293 @property 294 def _seen_error(self): 295 return self._last_error is not None 296 297 def check_output(self, want, got, optionflags): 298 # When an error occurs check_output is called with want=example.exc_msg, 299 # and got=exc_msg 300 301 # First check if `want` is a special string indicating wont_implement_ok 302 # and/or not_implemented_ok 303 allowed_exceptions = want.split('|') 304 if all(exc in (WONT_IMPLEMENT, NOT_IMPLEMENTED) 305 for exc in allowed_exceptions): 306 # If it is, check for WontImplementError and NotImplementedError 307 if WONT_IMPLEMENT in allowed_exceptions and got.startswith( 308 WONT_IMPLEMENT): 309 self._last_error = WONT_IMPLEMENT 310 return True 311 312 elif NOT_IMPLEMENTED in allowed_exceptions and got.startswith( 313 NOT_IMPLEMENTED): 314 self._last_error = NOT_IMPLEMENTED 315 return True 316 317 elif got.startswith('NameError') and self._seen_error: 318 # This allows us to gracefully skip tests like 319 # >>> res = df.unsupported_operation() 320 # >>> check(res) 321 return True 322 323 self.reset() 324 want, got = self.fix(want, got) 325 return super().check_output(want, got, optionflags) 326 327 def output_difference(self, example, got, optionflags): 328 want, got = self.fix(example.want, got) 329 if want != example.want: 330 example = doctest.Example( 331 example.source, 332 want, 333 example.exc_msg, 334 example.lineno, 335 example.indent, 336 example.options) 337 return super().output_difference(example, got, optionflags) 338 339 340 class BeamDataframeDoctestRunner(doctest.DocTestRunner): 341 """A Doctest runner suitable for replacing the `pd` module with one backed 342 by beam. 343 """ 344 def __init__( 345 self, 346 env, 347 use_beam=True, 348 wont_implement_ok=None, 349 not_implemented_ok=None, 350 skip=None, 351 **kwargs): 352 self._test_env = env 353 354 def to_callable(cond): 355 if cond == '*': 356 return lambda example: True 357 else: 358 return lambda example: example.source.strip() == cond 359 360 self._wont_implement_ok = { 361 test: [to_callable(cond) for cond in examples] 362 for test, 363 examples in (wont_implement_ok or {}).items() 364 } 365 self._not_implemented_ok = { 366 test: [to_callable(cond) for cond in examples] 367 for test, 368 examples in (not_implemented_ok or {}).items() 369 } 370 self._skip = { 371 test: [to_callable(cond) for cond in examples] 372 for test, 373 examples in (skip or {}).items() 374 } 375 super().__init__( 376 checker=_DeferrredDataframeOutputChecker(self._test_env, use_beam), 377 **kwargs) 378 self.success = 0 379 self.skipped = 0 380 self._reasons = collections.defaultdict(list) 381 self._skipped_set = set() 382 383 def _is_wont_implement_ok(self, example, test): 384 return any( 385 wont_implement(example) 386 for wont_implement in self._wont_implement_ok.get(test.name, [])) 387 388 def _is_not_implemented_ok(self, example, test): 389 return any( 390 not_implemented(example) 391 for not_implemented in self._not_implemented_ok.get(test.name, [])) 392 393 def run(self, test, **kwargs): 394 self._checker.reset() 395 for example in test.examples: 396 if any(should_skip(example) 397 for should_skip in self._skip.get(test.name, [])): 398 self._skipped_set.add(example) 399 example.source = 'pass' 400 example.want = '' 401 self.skipped += 1 402 elif example.exc_msg is None: 403 allowed_exceptions = [] 404 if self._is_not_implemented_ok(example, test): 405 allowed_exceptions.append(NOT_IMPLEMENTED) 406 if self._is_wont_implement_ok(example, test): 407 allowed_exceptions.append(WONT_IMPLEMENT) 408 409 if len(allowed_exceptions): 410 # Don't fail doctests that raise this error. 411 example.exc_msg = '|'.join(allowed_exceptions) 412 with self._test_env.context(): 413 result = super().run(test, **kwargs) 414 # Can't add attributes to builtin result. 415 result = AugmentedTestResults(result.failed, result.attempted) 416 result.summary = self.summary() 417 return result 418 419 def report_success(self, out, test, example, got): 420 def extract_concise_reason(got, expected_exc): 421 m = re.search(r"Implement(?:ed)?Error:\s+(.*)\n$", got) 422 if m: 423 return m.group(1) 424 elif "NameError" in got: 425 return "NameError following %s" % expected_exc 426 elif re.match(r"DeferredBase\[\d+\]\n", got): 427 return "DeferredBase[*]" 428 else: 429 return got.replace("\n", "\\n") 430 431 if self._checker._last_error is not None: 432 self._reasons[self._checker._last_error].append( 433 extract_concise_reason(got, self._checker._last_error)) 434 435 if self._checker._seen_error: 436 m = re.search('^([a-zA-Z0-9_, ]+)=', example.source) 437 if m: 438 for var in m.group(1).split(','): 439 var = var.strip() 440 if var in test.globs: 441 # More informative to get a NameError than 442 # use the wrong previous value. 443 del test.globs[var] 444 445 return super().report_success(out, test, example, got) 446 447 def fake_pandas_module(self): 448 return self._test_env.fake_pandas_module() 449 450 def summarize(self): 451 super().summarize() 452 self.summary().summarize() 453 454 def summary(self): 455 return Summary(self.failures, self.tries, self.skipped, self._reasons) 456 457 458 class AugmentedTestResults(doctest.TestResults): 459 pass 460 461 462 class Summary(object): 463 def __init__(self, failures=0, tries=0, skipped=0, error_reasons=None): 464 self.failures = failures 465 self.tries = tries 466 self.skipped = skipped 467 self.error_reasons = error_reasons or collections.defaultdict(list) 468 469 def result(self): 470 res = AugmentedTestResults(self.failures, self.tries) 471 res.summary = self 472 return res 473 474 def __add__(self, other): 475 merged_reasons = { 476 key: self.error_reasons.get(key, []) + other.error_reasons.get(key, []) 477 for key in set(self.error_reasons.keys()).union( 478 other.error_reasons.keys()) 479 } 480 return Summary( 481 self.failures + other.failures, 482 self.tries + other.tries, 483 self.skipped + other.skipped, 484 merged_reasons) 485 486 def summarize(self): 487 def print_partition(indent, desc, n, total): 488 print("%s%d %s (%.1f%%)" % (" " * indent, n, desc, n / total * 100)) 489 490 print() 491 print("%d total test cases:" % self.tries) 492 493 if not self.tries: 494 return 495 496 print_partition(1, "skipped", self.skipped, self.tries) 497 for error, reasons in self.error_reasons.items(): 498 print_partition(1, error, len(reasons), self.tries) 499 reason_counts = sorted( 500 collections.Counter(reasons).items(), 501 key=lambda x: x[1], 502 reverse=True) 503 for desc, count in reason_counts: 504 print_partition(2, desc, count, len(reasons)) 505 print_partition(1, "failed", self.failures, self.tries) 506 print_partition( 507 1, 508 "passed", 509 self.tries - self.skipped - 510 sum(len(reasons) 511 for reasons in self.error_reasons.values()) - self.failures, 512 self.tries) 513 print() 514 515 516 def parse_rst_ipython_tests(rst, name, extraglobs=None, optionflags=None): 517 """Extracts examples from an rst file and produce a test suite by running 518 them through pandas to get the expected outputs. 519 """ 520 521 # Optional dependency. 522 import IPython 523 from traitlets.config import Config 524 525 def get_indent(line): 526 return len(line) - len(line.lstrip()) 527 528 def is_example_line(line): 529 line = line.strip() 530 return line and not line.startswith('#') and not line[0] == line[-1] == ':' 531 532 IMPORT_PANDAS = 'import pandas as pd' 533 534 example_srcs = [] 535 lines = iter([(lineno, line.rstrip()) for lineno, 536 line in enumerate(rst.split('\n')) if is_example_line(line)] + 537 [(None, 'END')]) 538 539 # https://ipython.readthedocs.io/en/stable/sphinxext.html 540 lineno, line = next(lines) 541 while True: 542 if line == 'END': 543 break 544 if line.startswith('.. ipython::'): 545 lineno, line = next(lines) 546 indent = get_indent(line) 547 example = [] 548 example_srcs.append((lineno, example)) 549 while get_indent(line) >= indent: 550 if '@verbatim' in line or ':verbatim:' in line or '@savefig' in line: 551 example_srcs.pop() 552 break 553 line = re.sub(r'In \[\d+\]: ', '', line) 554 line = re.sub(r'\.\.\.+:', '', line) 555 example.append(line[indent:]) 556 lineno, line = next(lines) 557 if get_indent(line) == indent and line[indent] not in ')]}': 558 example = [] 559 example_srcs.append((lineno, example)) 560 else: 561 lineno, line = next(lines) 562 563 # TODO(robertwb): Would it be better to try and detect/compare the actual 564 # objects in two parallel sessions than make (stringified) doctests? 565 examples = [] 566 567 config = Config() 568 config.HistoryManager.hist_file = ':memory:' 569 config.InteractiveShell.autocall = False 570 config.InteractiveShell.autoindent = False 571 config.InteractiveShell.colors = 'NoColor' 572 573 set_pandas_options() 574 IP = IPython.InteractiveShell.instance(config=config) 575 IP.run_cell(IMPORT_PANDAS + '\n') 576 IP.run_cell('import numpy as np\n') 577 try: 578 stdout = sys.stdout 579 for lineno, src in example_srcs: 580 sys.stdout = cout = StringIO() 581 src = '\n'.join(src) 582 if src == IMPORT_PANDAS: 583 continue 584 IP.run_cell(src + '\n') 585 output = cout.getvalue() 586 if output: 587 # Strip the prompt. 588 # TODO(robertwb): Figure out how to suppress this. 589 output = re.sub(r'^Out\[\d+\]:[ \t]*\n?', '', output) 590 examples.append(doctest.Example(src, output, lineno=lineno)) 591 592 finally: 593 sys.stdout = stdout 594 595 return doctest.DocTest( 596 examples, dict(extraglobs or {}, np=np), name, name, None, None) 597 598 599 def test_rst_ipython( 600 rst, 601 name, 602 report=False, 603 wont_implement_ok=(), 604 not_implemented_ok=(), 605 skip=(), 606 **kwargs): 607 """Extracts examples from an rst file and run them through pandas to get the 608 expected output, and then compare them against our dataframe implementation. 609 """ 610 def run_tests(extraglobs, optionflags, **kwargs): 611 # The patched one. 612 tests = parse_rst_ipython_tests(rst, name, extraglobs, optionflags) 613 runner = doctest.DocTestRunner(optionflags=optionflags) 614 set_pandas_options() 615 result = runner.run(tests, **kwargs) 616 if report: 617 runner.summarize() 618 return result 619 620 result = _run_patched( 621 run_tests, 622 wont_implement_ok={name: wont_implement_ok}, 623 not_implemented_ok={name: not_implemented_ok}, 624 skip={name: skip}, 625 **kwargs) 626 return result 627 628 629 def teststring(text, wont_implement_ok=None, not_implemented_ok=None, **kwargs): 630 return teststrings( 631 {'<string>': text}, 632 wont_implement_ok={'<string>': ['*']} if wont_implement_ok else None, 633 not_implemented_ok={'<string>': ['*']} if not_implemented_ok else None, 634 **kwargs) 635 636 637 def teststrings(texts, report=False, **runner_kwargs): 638 optionflags = runner_kwargs.pop('optionflags', 0) 639 optionflags |= ( 640 doctest.NORMALIZE_WHITESPACE | doctest.IGNORE_EXCEPTION_DETAIL) 641 642 parser = doctest.DocTestParser() 643 runner = BeamDataframeDoctestRunner( 644 TestEnvironment(), optionflags=optionflags, **runner_kwargs) 645 globs = { 646 'pd': runner.fake_pandas_module(), 647 'np': np, 648 'option_context': pd.option_context, 649 } 650 with expressions.allow_non_parallel_operations(): 651 for name, text in texts.items(): 652 test = parser.get_doctest(text, globs, name, name, 0) 653 runner.run(test) 654 if report: 655 runner.summarize() 656 return runner.summary().result() 657 658 659 def set_pandas_options(): 660 # See 661 # https://github.com/pandas-dev/pandas/blob/a00202d12d399662b8045a8dd3fdac04f18e1e55/doc/source/conf.py#L319 662 np.random.seed(123456) 663 np.set_printoptions(precision=4, suppress=True) 664 pd.options.display.max_rows = 15 665 666 667 def _run_patched(func, *args, **kwargs): 668 set_pandas_options() 669 670 # https://github.com/pandas-dev/pandas/blob/1.0.x/setup.cfg#L63 671 optionflags = kwargs.pop('optionflags', 0) 672 optionflags |= ( 673 doctest.NORMALIZE_WHITESPACE | doctest.IGNORE_EXCEPTION_DETAIL) 674 675 env = TestEnvironment() 676 use_beam = kwargs.pop('use_beam', True) 677 skip = kwargs.pop('skip', {}) 678 wont_implement_ok = kwargs.pop('wont_implement_ok', {}) 679 not_implemented_ok = kwargs.pop('not_implemented_ok', {}) 680 extraglobs = dict(kwargs.pop('extraglobs', {})) 681 extraglobs['pd'] = env.fake_pandas_module() 682 683 try: 684 # Unfortunately the runner is not injectable. 685 original_doc_test_runner = doctest.DocTestRunner 686 doctest.DocTestRunner = lambda **kwargs: BeamDataframeDoctestRunner( 687 env, 688 use_beam=use_beam, 689 wont_implement_ok=wont_implement_ok, 690 not_implemented_ok=not_implemented_ok, 691 skip=skip, 692 **kwargs) 693 with expressions.allow_non_parallel_operations(): 694 return func( 695 *args, extraglobs=extraglobs, optionflags=optionflags, **kwargs) 696 finally: 697 doctest.DocTestRunner = original_doc_test_runner 698 699 700 def with_run_patched_docstring(target=None): 701 assert target is not None 702 703 def wrapper(fn): 704 fn.__doc__ = f"""Run all pandas doctests in the specified {target}. 705 706 Arguments `skip`, `wont_implement_ok`, `not_implemented_ok` are all in the 707 format:: 708 709 {{ 710 "module.Class.method": ['*'], 711 "module.Class.other_method": [ 712 'instance.other_method(bad_input)', 713 'observe_result_of_bad_input()', 714 ], 715 }} 716 717 `'*'` indicates all examples should be matched, otherwise the list is a list 718 of specific input strings that should be matched. 719 720 All arguments are kwargs. 721 722 Args: 723 optionflags (int): Passed through to doctests. 724 extraglobs (Dict[str,Any]): Passed through to doctests. 725 use_beam (bool): If true, run a Beam pipeline with partitioned input to 726 verify the examples, else use PartitioningSession to simulate 727 distributed execution. 728 skip (Dict[str,str]): A set of examples to skip entirely. 729 wont_implement_ok (Dict[str,str]): A set of examples that are allowed to 730 raise WontImplementError. 731 not_implemented_ok (Dict[str,str]): A set of examples that are allowed to 732 raise NotImplementedError. 733 734 Returns: 735 ~doctest.TestResults: A doctest result describing the passed/failed tests. 736 """ 737 return fn 738 739 return wrapper 740 741 742 @with_run_patched_docstring(target="file") 743 def testfile(*args, **kwargs): 744 return _run_patched(doctest.testfile, *args, **kwargs) 745 746 747 @with_run_patched_docstring(target="module") 748 def testmod(*args, **kwargs): 749 return _run_patched(doctest.testmod, *args, **kwargs)