Commit e7d8ca72 authored by Stephan Richter's avatar Stephan Richter

Checkpoint of Python 3 port. Over 800 of the 897 unit tests are passing.

Thanks to Martin v. Loewis for having done some of the initial lifting
for a DZUG talk. I used a lot of his changes from this diff:

http://www.dcl.hpi.uni-potsdam.de/home/loewis/zodb/ZODB.diff

Applied all changes manually to make sure I do understand them and that
they are correct.

As always 90% of the work is to catch all places where we want to deal
with bytes instead of strings.
parent 2ad61dba
......@@ -18,49 +18,148 @@ The script accepts buildout command-line options, so you can
use the -c option to specify an alternate configuration file.
"""
import os, shutil, sys, tempfile, urllib2
import os, shutil, sys, tempfile
from optparse import OptionParser
tmpeggs = tempfile.mkdtemp()
ez = {}
exec urllib2.urlopen('http://peak.telecommunity.com/dist/ez_setup.py'
).read() in ez
ez['use_setuptools'](to_dir=tmpeggs, download_delay=0)
usage = '''\
[DESIRED PYTHON FOR BUILDOUT] bootstrap.py [options]
import pkg_resources
Bootstraps a buildout-based project.
is_jython = sys.platform.startswith('java')
Simply run this script in a directory containing a buildout.cfg, using the
Python that you want bin/buildout to use.
if is_jython:
import subprocess
Note that by using --setup-source and --download-base to point to
local resources, you can keep this script from going over the network.
'''
cmd = 'from setuptools.command.easy_install import main; main()'
if sys.platform == 'win32':
cmd = '"%s"' % cmd # work around spawn lamosity on windows
parser = OptionParser(usage=usage)
parser.add_option("-v", "--version", help="use a specific zc.buildout version")
ws = pkg_resources.working_set
parser.add_option("-t", "--accept-buildout-test-releases",
dest='accept_buildout_test_releases',
action="store_true", default=False,
help=("Normally, if you do not specify a --version, the "
"bootstrap script and buildout gets the newest "
"*final* versions of zc.buildout and its recipes and "
"extensions for you. If you use this flag, "
"bootstrap and buildout will get the newest releases "
"even if they are alphas or betas."))
parser.add_option("-c", "--config-file",
help=("Specify the path to the buildout configuration "
"file to be used."))
parser.add_option("-f", "--find-links",
help=("Specify a URL to search for buildout releases"))
if is_jython:
assert subprocess.Popen(
[sys.executable] + ['-c', cmd, '-mqNxd', tmpeggs, 'zc.buildout'],
env = dict(os.environ,
PYTHONPATH=
ws.find(pkg_resources.Requirement.parse('setuptools')).location
),
).wait() == 0
else:
assert os.spawnle(
os.P_WAIT, sys.executable, sys.executable,
'-c', cmd, '-mqNxd', tmpeggs, 'zc.buildout',
dict(os.environ,
PYTHONPATH=
ws.find(pkg_resources.Requirement.parse('setuptools')).location
),
) == 0
options, args = parser.parse_args()
######################################################################
# load/install distribute
to_reload = False
try:
import pkg_resources, setuptools
if not hasattr(pkg_resources, '_distribute'):
to_reload = True
raise ImportError
except ImportError:
ez = {}
try:
from urllib.request import urlopen
except ImportError:
from urllib2 import urlopen
exec(urlopen('http://python-distribute.org/distribute_setup.py').read(), ez)
setup_args = dict(to_dir=tmpeggs, download_delay=0, no_fake=True)
ez['use_setuptools'](**setup_args)
if to_reload:
reload(pkg_resources)
import pkg_resources
# This does not (always?) update the default working set. We will
# do it.
for path in sys.path:
if path not in pkg_resources.working_set.entries:
pkg_resources.working_set.add_entry(path)
######################################################################
# Install buildout
ws = pkg_resources.working_set
cmd = [sys.executable, '-c',
'from setuptools.command.easy_install import main; main()',
'-mZqNxd', tmpeggs]
find_links = os.environ.get(
'bootstrap-testing-find-links',
options.find_links or
('http://downloads.buildout.org/'
if options.accept_buildout_test_releases else None)
)
if find_links:
cmd.extend(['-f', find_links])
distribute_path = ws.find(
pkg_resources.Requirement.parse('distribute')).location
requirement = 'zc.buildout'
version = options.version
if version is None and not options.accept_buildout_test_releases:
# Figure out the most recent final version of zc.buildout.
import setuptools.package_index
_final_parts = '*final-', '*final'
def _final_version(parsed_version):
for part in parsed_version:
if (part[:1] == '*') and (part not in _final_parts):
return False
return True
index = setuptools.package_index.PackageIndex(
search_path=[distribute_path])
if find_links:
index.add_find_links((find_links,))
req = pkg_resources.Requirement.parse(requirement)
if index.obtain(req) is not None:
best = []
bestv = None
for dist in index[req.project_name]:
distv = dist.parsed_version
if _final_version(distv):
if bestv is None or distv > bestv:
best = [dist]
bestv = distv
elif distv == bestv:
best.append(dist)
if best:
best.sort()
version = best[-1].version
if version:
requirement = '=='.join((requirement, version))
cmd.append(requirement)
import subprocess
if subprocess.call(cmd, env=dict(os.environ, PYTHONPATH=distribute_path)) != 0:
raise Exception(
"Failed to execute command:\n%s",
repr(cmd)[1:-1])
######################################################################
# Import and run buildout
ws.add_entry(tmpeggs)
ws.require('zc.buildout')
ws.require(requirement)
import zc.buildout.buildout
zc.buildout.buildout.main(sys.argv[1:] + ['bootstrap'])
if not [a for a in args if '=' not in a]:
args.append('bootstrap')
# if -c was provided, we push it back into args for buildout' main function
if options.config_file is not None:
args[0:0] = ['-c', options.config_file]
zc.buildout.buildout.main(args)
shutil.rmtree(tmpeggs)
[buildout]
develop = .
../zdaemon
parts =
test
scripts
versions = versions
[versions]
zc.recipe.testrunner = 1.3.0
zc.recipe.testrunner = 2.0.0
zc.lockfile = 1.1.0
zope.interface = 4.0.3
ZConfig = 3.0.1
BTrees = 4.0.5
persistent = 4.0.6
transaction = 1.4.0
[test]
recipe = zc.recipe.testrunner
......@@ -19,6 +26,23 @@ initialization =
tempfile.tempdir = os.path.abspath('tmp')
defaults = ['--all']
[coverage-test]
recipe = zc.recipe.testrunner
eggs = ${test:eggs}
initialization =
import os, tempfile
try: os.mkdir('tmp')
except: pass
tempfile.tempdir = os.path.abspath('tmp')
defaults = ['--coverage', '${buildout:directory}/coverage']
[coverage-report]
recipe = zc.recipe.egg
eggs = z3c.coverage
scripts = coveragereport=coverage-report
arguments = ('${buildout:directory}/coverage',
'${buildout:directory}/coverage/report')
[scripts]
recipe = zc.recipe.egg
eggs = ${test:eggs}
......
......@@ -22,16 +22,12 @@ interface, rich transaction support, and undo.
VERSION = "4.0.0dev"
from ez_setup import use_setuptools
use_setuptools()
from setuptools import setup, find_packages
from setuptools.extension import Extension
import os
import sys
from setuptools import setup, find_packages
if sys.version_info < (2, 6):
print "This version of ZODB requires Python 2.6 or higher"
print("This version of ZODB requires Python 2.6 or higher")
sys.exit(0)
# The (non-obvious!) choices for the Trove Development Status line:
......@@ -96,14 +92,15 @@ doclines = __doc__.split("\n")
def read_file(*path):
base_dir = os.path.dirname(__file__)
file_path = (base_dir, ) + tuple(path)
return file(os.path.join(*file_path)).read()
with open(os.path.join(*file_path), 'rb') as file:
return file.read()
long_description = str(
("\n".join(doclines[2:]) + "\n\n" +
".. contents::\n\n" +
read_file("README.rst") + "\n\n" +
read_file("CHANGES.rst")
).decode('latin-1').replace(u'L\xf6wis', '|Lowis|')
read_file("README.rst").decode('latin-1') + "\n\n" +
read_file("CHANGES.rst").decode('latin-1')
).replace(u'L\xf6wis', '|Lowis|')
)+ '''\n\n.. |Lowis| unicode:: L \\xf6 wis\n'''
tests_require = ['zope.testing', 'manuel']
......@@ -124,11 +121,12 @@ setup(name="ZODB",
tests_require = tests_require,
extras_require = dict(test=tests_require),
install_requires = [
'transaction',
'persistent',
'BTrees',
'zc.lockfile',
'ZConfig',
'persistent',
'transaction',
'six',
'zc.lockfile',
'zdaemon',
'zope.interface',
],
......
......@@ -16,10 +16,8 @@
The base class here is tightly coupled with its subclasses and
its use is not recommended. It's still here for historical reasons.
"""
from __future__ import print_function, with_statement
from __future__ import with_statement
import cPickle
import threading
import time
import logging
......@@ -31,9 +29,16 @@ from persistent.TimeStamp import TimeStamp
import ZODB.interfaces
from ZODB import POSException
from ZODB.utils import z64, oid_repr
from ZODB.utils import z64, oid_repr, byte_ord, byte_chr
from ZODB.UndoLogCompatible import UndoLogCompatible
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
log = logging.getLogger("ZODB.BaseStorage")
import sys
......@@ -173,9 +178,9 @@ class BaseStorage(UndoLogCompatible):
self._lock_acquire()
try:
last = self._oid
d = ord(last[-1])
d = byte_ord(last[-1])
if d < 255: # fast path for the usual case
last = last[:-1] + chr(d+1)
last = last[:-1] + byte_chr(d+1)
else: # there's a carry out of the last byte
last_as_long, = _structunpack(">Q", last)
last = _structpack(">Q", last_as_long + 1)
......@@ -237,7 +242,7 @@ class BaseStorage(UndoLogCompatible):
desc = transaction.description
ext = transaction._extension
if ext:
ext = cPickle.dumps(ext, 1)
ext = pickle.dumps(ext, 1)
else:
ext = ""
......@@ -377,24 +382,24 @@ def copy(source, dest, verbose=0):
else:
t = TimeStamp(tid)
if t <= _ts:
if ok: print ('Time stamps out of order %s, %s' % (_ts, t))
if ok: print(('Time stamps out of order %s, %s' % (_ts, t)))
ok = 0
_ts = t.laterThan(_ts)
tid = _ts.raw()
else:
_ts = t
if not ok:
print ('Time stamps back in order %s' % (t))
print(('Time stamps back in order %s' % (t)))
ok = 1
if verbose:
print _ts
print(_ts)
dest.tpc_begin(transaction, tid, transaction.status)
for r in transaction:
oid = r.oid
if verbose:
print oid_repr(oid), r.version, len(r.data)
print(oid_repr(oid), r.version, len(r.data))
if restoring:
dest.restore(oid, r.tid, r.data, r.version,
r.data_txn, transaction)
......
......@@ -13,14 +13,26 @@
##############################################################################
import logging
from cStringIO import StringIO
from cPickle import Unpickler, Pickler
from pickle import PicklingError
import zope.interface
from ZODB.POSException import ConflictError
from ZODB.loglevels import BLATHER
from ZODB.serialize import _protocol
try:
from cStringIO import StringIO
except ImportError:
# Py3
from io import StringIO
try:
from cPickle import Unpickler, Pickler
except ImportError:
# Py3
from pickle import Unpickler, Pickler
logger = logging.getLogger('ZODB.ConflictResolution')
......@@ -130,7 +142,7 @@ class PersistentReference(object):
# it. Fortunately, a class reference in a persistent
# reference is allowed to be a module+name tuple.
self.data = self.oid, klass.args
elif isinstance(data, str):
elif isinstance(data, bytes):
self.oid = data
else: # a list
reference_type = data[0]
......@@ -258,8 +270,11 @@ def tryToResolveConflict(self, oid, committedSerial, oldSerial, newpickle,
resolved = resolve(old, committed, newstate)
file = StringIO()
pickler = Pickler(file,1)
pickler.inst_persistent_id = persistent_id
pickler = Pickler(file, _protocol)
if sys.version_info[0] < 3:
pickler.inst_persistent_id = persistent_id
else:
pickler.inst_persistent_id = persistent_id
pickler.dump(meta)
pickler.dump(resolved)
return self._crs_transform_record_data(file.getvalue(1))
......
......@@ -48,8 +48,9 @@ from ZODB.POSException import ConflictError, ReadConflictError
from ZODB.POSException import Unsupported, ReadOnlyHistoryError
from ZODB.POSException import POSKeyError
from ZODB.serialize import ObjectWriter, ObjectReader
from ZODB.utils import p64, u64, z64, oid_repr, positive_id
from ZODB.utils import p64, u64, z64, oid_repr, positive_id, bytes
from ZODB import utils
import six
global_reset_counter = 0
......@@ -260,14 +261,14 @@ class Connection(ExportImport, object):
def cacheMinimize(self):
"""Deactivate all unmodified objects in the cache.
"""
for connection in self.connections.itervalues():
for connection in six.itervalues(self.connections):
connection._cache.minimize()
# TODO: we should test what happens when cacheGC is called mid-transaction.
def cacheGC(self):
"""Reduce cache size to target size.
"""
for connection in self.connections.itervalues():
for connection in six.itervalues(self.connections):
connection._cache.incrgc()
__onCloseCallbacks = None
......@@ -566,7 +567,7 @@ class Connection(ExportImport, object):
else:
self._commit(transaction)
for oid, serial in self._readCurrent.iteritems():
for oid, serial in six.iteritems(self._readCurrent):
try:
self._storage.checkCurrentSerialInTransaction(
oid, serial, transaction)
......@@ -708,7 +709,7 @@ class Connection(ExportImport, object):
if not serial:
return
if not isinstance(serial, str):
if not isinstance(serial, bytes):
raise serial
obj = self._cache.get(oid, None)
if obj is None:
......@@ -779,7 +780,7 @@ class Connection(ExportImport, object):
return
try:
s = vote(transaction)
except ReadConflictError, v:
except ReadConflictError as v:
if v.oid:
self._cache.invalidate(v.oid)
raise
......@@ -1070,7 +1071,7 @@ class Connection(ExportImport, object):
self._reader._cache = cache
def _release_resources(self):
for c in self.connections.itervalues():
for c in six.itervalues(self.connections):
if c._mvcc_storage:
c._storage.release()
c._storage = c._normal_storage = None
......@@ -1248,7 +1249,7 @@ class TmpStore:
self._file = tempfile.TemporaryFile()
# position: current file position
# _tpos: file position at last commit point
self.position = 0L
self.position = 0
# index: map oid to pos of last committed version
self.index = {}
self.creating = {}
......@@ -1301,7 +1302,7 @@ class TmpStore:
targetpath = self._getBlobPath()
if not os.path.exists(targetpath):
os.makedirs(targetpath, 0700)
os.makedirs(targetpath, 0o700)
targetname = self._getCleanFilename(oid, serial)
rename_or_copy_blob(blobfilename, targetname, chmod=False)
......
......@@ -13,9 +13,6 @@
##############################################################################
"""Database objects
"""
import cPickle
import cStringIO
import sys
import threading
import logging
......@@ -37,7 +34,19 @@ from ZODB.interfaces import IMVCCStorage
import transaction
from persistent.TimeStamp import TimeStamp
import six
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
try:
from cStringIO import StringIO as BytesIO
except ImportError:
# Py3
from io import BytesIO
logger = logging.getLogger('ZODB.DB')
......@@ -271,7 +280,7 @@ class KeyedConnectionPool(AbstractConnectionPool):
return pool.pop()
def map(self, f):
for pool in self.pools.itervalues():
for pool in six.itervalues(self.pools):
pool.map(f)
def availableGC(self):
......@@ -283,14 +292,14 @@ class KeyedConnectionPool(AbstractConnectionPool):
@property
def test_all(self):
result = set()
for pool in self.pools.itervalues():
for pool in six.itervalues(self.pools):
result.update(pool.all)
return frozenset(result)
@property
def test_available(self):
result = []
for pool in self.pools.itervalues():
for pool in six.itervalues(self.pools):
result.extend(pool.available)
return tuple(result)
......@@ -398,7 +407,7 @@ class DB(object):
- `xrefs` - Boolian flag indicating whether implicit cross-database
references are allowed
"""
if isinstance(storage, basestring):
if isinstance(storage, six.string_types):
from ZODB import FileStorage
storage = ZODB.FileStorage.FileStorage(storage, **storage_args)
elif storage is None:
......@@ -448,8 +457,8 @@ class DB(object):
root = PersistentMapping()
# Manually create a pickle for the root to put in the storage.
# The pickle must be in the special ZODB format.
file = cStringIO.StringIO()
p = cPickle.Pickler(file, 1)
file = BytesIO()
p = pickle.Pickler(file, ZODB.serialize._protocol)
p.dump((root.__class__, None))
p.dump(root.__getstate__())
t = transaction.Transaction()
......@@ -540,9 +549,9 @@ class DB(object):
id = ''
if hasattr(ob, '__dict__'):
d = ob.__dict__
if d.has_key('id'):
if 'id' in d:
id = d['id']
elif d.has_key('__name__'):
elif '__name__' in d:
id = d['__name__']
module = getattr(ob.__class__, '__module__', '')
......@@ -713,7 +722,7 @@ class DB(object):
raise ValueError(
'cannot open an historical connection in the future.')
if isinstance(transaction_manager, basestring):
if isinstance(transaction_manager, six.string_types):
if transaction_manager:
raise TypeError("Versions aren't supported.")
warnings.warn(
......
......@@ -249,7 +249,7 @@ class DemoStorage(object):
try:
self.changes.pack(t, referencesf, gc=False)
except TypeError, v:
except TypeError as v:
if 'gc' in str(v):
pass # The gc arg isn't supported. Don't pack
raise
......
......@@ -12,20 +12,30 @@
#
##############################################################################
"""Support for database export and import."""
import os
from cStringIO import StringIO
from cPickle import Pickler, Unpickler
from tempfile import TemporaryFile
import logging
import six
from ZODB.blob import Blob
from ZODB.interfaces import IBlobStorage
from ZODB.POSException import ExportError
from ZODB.serialize import referencesf
from ZODB.serialize import referencesf, _protocol
from ZODB.utils import p64, u64, cp, mktemp
try:
from cStringIO import StringIO as BytesIO
except ImportError:
# Py3
from io import BytesIO
try:
from cPickle import Unpickler, Pickler
except ImportError:
# Py3
from pickle import Unpickler, Pickler
logger = logging.getLogger('ZODB.ExportImport')
class ExportImport:
......@@ -33,12 +43,12 @@ class ExportImport:
def exportFile(self, oid, f=None):
if f is None:
f = TemporaryFile()
elif isinstance(f, str):
elif isinstance(f, six.string_types):
f = open(f,'w+b')
f.write('ZEXP')
f.write(b'ZEXP')
oids = [oid]
done_oids = {}
done=done_oids.has_key
done=done_oids.__contains__
load=self._storage.load
supports_blobs = IBlobStorage.providedBy(self._storage)
while oids:
......@@ -72,12 +82,12 @@ class ExportImport:
def importFile(self, f, clue='', customImporters=None):
# This is tricky, because we need to work in a transaction!
if isinstance(f, str):
if isinstance(f, six.string_types):
f = open(f, 'rb')
magic = f.read(4)
if magic != 'ZEXP':
if customImporters and customImporters.has_key(magic):
if magic != b'ZEXP':
if customImporters and magic in customImporters:
f.seek(0)
return customImporters[magic](self, f, clue)
raise ExportError("Invalid export header")
......@@ -164,13 +174,16 @@ class ExportImport:
f.seek(-len(blob_begin_marker),1)
blob_filename = None
pfile = StringIO(data)
pfile = BytesIO(data)
unpickler = Unpickler(pfile)
unpickler.persistent_load = persistent_load
newp = StringIO()
pickler = Pickler(newp, 1)
pickler.inst_persistent_id = persistent_id
newp = BytesIO()
pickler = Pickler(newp, _protocol)
if sys.version_info[0] < 3:
pickler.inst_persistent_id = persistent_id
else:
pickler.persistent_id = persistent_id
pickler.dump(unpickler.load())
pickler.dump(unpickler.load())
......@@ -183,8 +196,8 @@ class ExportImport:
self._storage.store(oid, None, data, '', transaction)
export_end_marker = '\377'*16
blob_begin_marker = '\000BLOBSTART'
export_end_marker = b'\377'*16
blob_begin_marker = b'\000BLOBSTART'
class Ghost(object):
__slots__ = ("oid",)
......
This diff is collapsed.
......@@ -83,11 +83,12 @@
# When we undo a record, we don't copy (or delete)
# data. Instead, we write records with back pointers.
import struct
import logging
import struct
import sys
from ZODB.POSException import POSKeyError
from ZODB.utils import u64, oid_repr
from ZODB.utils import u64, oid_repr, as_bytes
class CorruptedError(Exception):
......@@ -126,7 +127,7 @@ class FileStorageFormatter(object):
# subclasses must provide _file
_metadata_size = 4L
_metadata_size = 4
_format_version = "21"
def _read_num(self, pos):
......@@ -260,7 +261,10 @@ class DataHeader(object):
return DATA_HDR_LEN + (self.plen or 8)
def TxnHeaderFromString(s):
return TxnHeader(*struct.unpack(TRANS_HDR, s))
res = TxnHeader(*struct.unpack(TRANS_HDR, s))
if sys.version_info[0] >= 3:
res.status = res.status.decode('ascii')
return res
class TxnHeader(object):
"""Header for a transaction record."""
......@@ -278,9 +282,9 @@ class TxnHeader(object):
assert elen >= 0
def asString(self):
s = struct.pack(TRANS_HDR, self.tid, self.tlen, self.status,
s = struct.pack(TRANS_HDR, self.tid, self.tlen, as_bytes(self.status),
self.ulen, self.dlen, self.elen)
return "".join(map(str, [s, self.user, self.descr, self.ext]))
return b"".join(map(as_bytes, [s, self.user, self.descr, self.ext]))
def headerlen(self):
return TRANS_HDR_LEN + self.ulen + self.dlen + self.elen
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
##############################################################################
#
# Copyright (c) 2003 Zope Foundation and Contributors.
......@@ -23,13 +47,13 @@ def fsdump(path, file=None, with_offset=1):
iter = FileIterator(path)
for i, trans in enumerate(iter):
if with_offset:
print >> file, ("Trans #%05d tid=%016x time=%s offset=%d" %
(i, u64(trans.tid), TimeStamp(trans.tid), trans._pos))
print(("Trans #%05d tid=%016x time=%s offset=%d" %
(i, u64(trans.tid), TimeStamp(trans.tid), trans._pos)), file=file)
else:
print >> file, ("Trans #%05d tid=%016x time=%s" %
(i, u64(trans.tid), TimeStamp(trans.tid)))
print >> file, (" status=%r user=%r description=%r" %
(trans.status, trans.user, trans.description))
print(("Trans #%05d tid=%016x time=%s" %
(i, u64(trans.tid), TimeStamp(trans.tid))), file=file)
print((" status=%r user=%r description=%r" %
(trans.status, trans.user, trans.description)), file=file)
for j, rec in enumerate(trans):
if rec.data is None:
......@@ -47,8 +71,8 @@ def fsdump(path, file=None, with_offset=1):
else:
bp = ""
print >> file, (" data #%05d oid=%016x%s class=%s%s" %
(j, u64(rec.oid), size, fullclass, bp))
print((" data #%05d oid=%016x%s class=%s%s" %
(j, u64(rec.oid), size, fullclass, bp)), file=file)
iter.close()
def fmt(p64):
......@@ -66,8 +90,8 @@ class Dumper:
def dump(self):
fid = self.file.read(4)
print >> self.dest, "*" * 60
print >> self.dest, "file identifier: %r" % fid
print("*" * 60, file=self.dest)
print("file identifier: %r" % fid, file=self.dest)
while self.dump_txn():
pass
......@@ -78,12 +102,12 @@ class Dumper:
return False
tid, tlen, status, ul, dl, el = struct.unpack(TRANS_HDR, h)
end = pos + tlen
print >> self.dest, "=" * 60
print >> self.dest, "offset: %d" % pos
print >> self.dest, "end pos: %d" % end
print >> self.dest, "transaction id: %s" % fmt(tid)
print >> self.dest, "trec len: %d" % tlen
print >> self.dest, "status: %r" % status
print("=" * 60, file=self.dest)
print("offset: %d" % pos, file=self.dest)
print("end pos: %d" % end, file=self.dest)
print("transaction id: %s" % fmt(tid), file=self.dest)
print("trec len: %d" % tlen, file=self.dest)
print("status: %r" % status, file=self.dest)
user = descr = extra = ""
if ul:
user = self.file.read(ul)
......@@ -91,13 +115,13 @@ class Dumper:
descr = self.file.read(dl)
if el:
extra = self.file.read(el)
print >> self.dest, "user: %r" % user
print >> self.dest, "description: %r" % descr
print >> self.dest, "len(extra): %d" % el
print("user: %r" % user, file=self.dest)
print("description: %r" % descr, file=self.dest)
print("len(extra): %d" % el, file=self.dest)
while self.file.tell() < end:
self.dump_data(pos)
stlen = self.file.read(8)
print >> self.dest, "redundant trec len: %d" % u64(stlen)
print("redundant trec len: %d" % u64(stlen), file=self.dest)
return 1
def dump_data(self, tloc):
......@@ -105,18 +129,18 @@ class Dumper:
h = self.file.read(DATA_HDR_LEN)
assert len(h) == DATA_HDR_LEN
oid, revid, prev, tloc, vlen, dlen = struct.unpack(DATA_HDR, h)
print >> self.dest, "-" * 60
print >> self.dest, "offset: %d" % pos
print >> self.dest, "oid: %s" % fmt(oid)
print >> self.dest, "revid: %s" % fmt(revid)
print >> self.dest, "previous record offset: %d" % prev
print >> self.dest, "transaction offset: %d" % tloc
print("-" * 60, file=self.dest)
print("offset: %d" % pos, file=self.dest)
print("oid: %s" % fmt(oid), file=self.dest)
print("revid: %s" % fmt(revid), file=self.dest)
print("previous record offset: %d" % prev, file=self.dest)
print("transaction offset: %d" % tloc, file=self.dest)
assert not vlen
print >> self.dest, "len(data): %d" % dlen
print("len(data): %d" % dlen, file=self.dest)
self.file.read(dlen)
if not dlen:
sbp = self.file.read(8)
print >> self.dest, "backpointer: %d" % u64(sbp)
print("backpointer: %d" % u64(sbp), file=self.dest)
def main():
import sys
......
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
##############################################################################
#
# Copyright (c) 2004 Zope Foundation and Contributors.
......@@ -16,6 +22,8 @@ import ZODB.FileStorage
from ZODB.utils import get_pickle_metadata, p64, oid_repr, tid_repr
from ZODB.serialize import get_refs
from ZODB.TimeStamp import TimeStamp
from six.moves import map
from six.moves import zip
# Extract module.class string from pickle.
def get_class(pickle):
......@@ -73,7 +81,7 @@ class Tracer(object):
entire database, including references.
"""
for oid in oids:
if isinstance(oid, str):
if isinstance(oid, bytes):
assert len(oid) == 8
else:
oid = p64(oid)
......@@ -104,22 +112,22 @@ class Tracer(object):
nrev = oids[oid]
revision = "revision" + (nrev != 1 and 's' or '')
name = oid2name.get(oid, "<unknown>")
print "oid", oid_repr(oid), name, nrev, revision
print("oid", oid_repr(oid), name, nrev, revision)
current_oid = oid
current_tid = None
if msg is NOT_SEEN:
assert tid is None
print " ", msg
print(" ", msg)
continue
if tid != current_tid:
current_tid = tid
status, user, description, pos = self.tid2info[tid]
print " tid %s offset=%d %s" % (tid_repr(tid),
print(" tid %s offset=%d %s" % (tid_repr(tid),
pos,
TimeStamp(tid))
print " tid user=%r" % shorten(user)
print " tid description=%r" % shorten(description)
print " ", msg
TimeStamp(tid)))
print(" tid user=%r" % shorten(user))
print(" tid description=%r" % shorten(description))
print(" ", msg)
# Do the analysis.
def run(self):
......
......@@ -28,6 +28,7 @@ from ZODB.FileStorage.format import DataHeader, TRANS_HDR_LEN
from ZODB.FileStorage.format import FileStorageFormatter, CorruptedDataError
from ZODB.utils import p64, u64, z64
import binascii
import logging
import os
import ZODB.fsIndex
......@@ -196,7 +197,7 @@ class GC(FileStorageFormatter):
self.reachable = self.oid2curpos
def buildPackIndex(self):
pos = 4L
pos = 4
# We make the initial assumption that the database has been
# packed before and set unpacked to True only after seeing the
# first record with a status == " ". If we get to the packtime
......@@ -241,7 +242,7 @@ class GC(FileStorageFormatter):
# packed earlier and the current pack is redudant.
try:
th = self._read_txn_header(pos)
except CorruptedDataError, err:
except CorruptedDataError as err:
if err.buf != "":
raise
if th.status == 'p':
......@@ -297,7 +298,7 @@ class GC(FileStorageFormatter):
self.checkData(th, tpos, dh, pos)
if dh.back and dh.back < self.packpos:
if self.reachable.has_key(dh.oid):
if dh.oid in self.reachable:
L = self.reach_ex.setdefault(dh.oid, [])
if dh.back not in L:
L.append(dh.back)
......@@ -343,7 +344,7 @@ class FileStoragePacker(FileStorageFormatter):
if storage.blob_dir:
self.pack_blobs = True
self.blob_removed = open(
os.path.join(storage.blob_dir, '.removed'), 'w')
os.path.join(storage.blob_dir, '.removed'), 'wb')
else:
self.pack_blobs = False
......@@ -453,7 +454,7 @@ class FileStoragePacker(FileStorageFormatter):
raise
def copyToPacktime(self):
offset = 0L # the amount of space freed by packing
offset = 0 # the amount of space freed by packing
pos = self._metadata_size
new_pos = pos
......@@ -489,7 +490,7 @@ class FileStoragePacker(FileStorageFormatter):
If any data records are copied, also write txn header (th).
"""
copy = 0
new_tpos = 0L
new_tpos = 0
tend = pos + th.tlen
pos += th.headerlen()
while pos < tend:
......@@ -515,10 +516,10 @@ class FileStoragePacker(FileStorageFormatter):
if not is_dup:
if h.oid not in self.gc.reachable:
self.blob_removed.write(
h.oid.encode('hex')+'\n')
binascii.hexlify(h.oid)+b'\n')
else:
self.blob_removed.write(
(h.oid+h.tid).encode('hex')+'\n')
binascii.hexlify(h.oid+h.tid)+b'\n')
pos += h.recordlen()
continue
......@@ -584,7 +585,7 @@ class FileStoragePacker(FileStorageFormatter):
try:
while 1:
ipos = self.copyOne(ipos)
except CorruptedDataError, err:
except CorruptedDataError as err:
# The last call to copyOne() will raise
# CorruptedDataError, because it will attempt to read past
# the end of the file. Double-check that the exception
......
......@@ -13,12 +13,23 @@
##############################################################################
import doctest
import os
import re
import time
import transaction
import unittest
import ZODB.blob
import ZODB.FileStorage
import ZODB.tests.util
from zope.testing import renormalizing
checker = renormalizing.RENormalizing([
# Python 3 bytes add a "b".
(re.compile("b('.*?')"), r"\1"),
# Python 3 adds module name to exceptions.
(re.compile("ZODB.POSException.POSKeyError"), r"POSKeyError"),
(re.compile("ZODB.FileStorage.FileStorage.FileStorageQuotaError"),
r"FileStorageQuotaError"),
])
def pack_keep_old():
"""Should a copy of the database be kept?
......@@ -30,11 +41,11 @@ The pack_keep_old constructor argument controls whether a .old file (and .old di
>>> conn = db.open()
>>> import ZODB.blob
>>> conn.root()[1] = ZODB.blob.Blob()
>>> conn.root()[1].open('w').write('some data')
>>> _ = conn.root()[1].open('w').write(b'some data')
>>> conn.root()[2] = ZODB.blob.Blob()
>>> conn.root()[2].open('w').write('some data')
>>> _ = conn.root()[2].open('w').write(b'some data')
>>> transaction.commit()
>>> conn.root()[1].open('w').write('some other data')
>>> _ = conn.root()[1].open('w').write(b'some other data')
>>> del conn.root()[2]
>>> transaction.commit()
>>> old_size = os.stat('data.fs').st_size
......@@ -66,11 +77,11 @@ The pack_keep_old constructor argument controls whether a .old file (and .old di
>>> db = ZODB.DB(fs)
>>> conn = db.open()
>>> conn.root()[1] = ZODB.blob.Blob()
>>> conn.root()[1].open('w').write('some data')
>>> _ = conn.root()[1].open('w').write(b'some data')
>>> conn.root()[2] = ZODB.blob.Blob()
>>> conn.root()[2].open('w').write('some data')
>>> _ = conn.root()[2].open('w').write(b'some data')
>>> transaction.commit()
>>> conn.root()[1].open('w').write('some other data')
>>> _ = conn.root()[1].open('w').write(b'some other data')
>>> del conn.root()[2]
>>> transaction.commit()
......@@ -106,7 +117,7 @@ def pack_with_repeated_blob_records():
>>> trans = tm.begin()
>>> fs.tpc_begin(trans)
>>> open('ablob', 'w').write('some data')
>>> _ = open('ablob', 'w').write('some data')
>>> _ = fs.store(oid, oldserial, blob_record, '', trans)
>>> _ = fs.storeBlob(oid, oldserial, blob_record, 'ablob', '', trans)
>>> fs.tpc_vote(trans)
......@@ -135,7 +146,7 @@ _save_index can fail for large indexes.
>>> oid = 0
>>> for i in range(5000):
... oid += (1<<16)
... _ = fs.store(ZODB.utils.p64(oid), ZODB.utils.z64, 'x', '', t)
... _ = fs.store(ZODB.utils.p64(oid), ZODB.utils.z64, b'x', '', t)
>>> fs.tpc_vote(t)
>>> fs.tpc_finish(t)
......@@ -171,9 +182,11 @@ def test_suite():
doctest.DocFileSuite(
'zconfig.txt', 'iterator.test',
setUp=ZODB.tests.util.setUp, tearDown=ZODB.tests.util.tearDown,
checker=checker
),
doctest.DocTestSuite(
setUp=ZODB.tests.util.setUp, tearDown=ZODB.tests.util.tearDown,
checker=checker
),
))
......@@ -64,7 +64,7 @@ create
Because the file was truncated, we no-longer have object 0:
>>> fs.load('\0'*8)
>>> fs.load(b'\0'*8)
Traceback (most recent call last):
...
POSKeyError: 0x00
......@@ -116,8 +116,9 @@ packer
To demonstrate this, we'll create a null packer that just prints
some information about it's arguments:
>>> import six
>>> def packer(storage, referencesf, stop, gc):
... print referencesf, storage is fs, gc, storage.pack_keep_old
... six.print_(referencesf, storage is fs, gc, storage.pack_keep_old)
>>> ZODB.FileStorage.config_demo_printing_packer = packer
>>> fs = ZODB.config.storageFromString("""
......@@ -140,8 +141,8 @@ packer
>>> def packer_factory(name):
... def packer(storage, referencesf, stop, gc):
... print repr(name), referencesf, storage is fs, gc,
... print storage.pack_keep_old
... six.print_(repr(name), referencesf, storage is fs,
... gc, storage.pack_keep_old)
... return packer
>>> ZODB.FileStorage.config_demo_printing_packer_factory = packer_factory
......
......@@ -34,7 +34,7 @@ def _recon(class_, state):
return err
_recon.__no_side_effects__ = True
class POSError(StandardError):
class POSError(Exception):
"""Persistent object system error."""
if sys.version_info[:2] == (2, 6):
......
......@@ -25,4 +25,4 @@ sys.modules['ZODB.PersistentList'] = sys.modules['persistent.list']
del mapping, list, sys
from DB import DB, connection
from ZODB.DB import DB, connection
......@@ -14,8 +14,6 @@
"""Blobs
"""
import cPickle
import cStringIO
import base64
import binascii
import logging
......@@ -31,10 +29,25 @@ import zope.interface
import ZODB.interfaces
from ZODB.interfaces import BlobError
from ZODB import utils
from ZODB import utils, serialize
from ZODB.POSException import POSKeyError
import persistent
try:
import cPickle
except ImportError:
# Py3
import pickle as cPickle
try:
from cStringIO import StringIO as BytesIO
except ImportError:
# Py3
from io import BytesIO
if sys.version_info[0] >= 3:
from io import FileIO as file
logger = logging.getLogger('ZODB.blob')
BLOB_SUFFIX = ".blob"
......@@ -337,18 +350,18 @@ class FilesystemHelper:
def create(self):
if not os.path.exists(self.base_dir):
os.makedirs(self.base_dir, 0700)
os.makedirs(self.base_dir, 0o700)
log("Blob directory '%s' does not exist. "
"Created new directory." % self.base_dir)
if not os.path.exists(self.temp_dir):
os.makedirs(self.temp_dir, 0700)
os.makedirs(self.temp_dir, 0o700)
log("Blob temporary directory '%s' does not exist. "
"Created new directory." % self.temp_dir)
if not os.path.exists(os.path.join(self.base_dir, LAYOUT_MARKER)):
layout_marker = open(
os.path.join(self.base_dir, LAYOUT_MARKER), 'wb')
layout_marker.write(self.layout_name)
layout_marker.write(utils.as_bytes(self.layout_name))
else:
layout = open(os.path.join(self.base_dir, LAYOUT_MARKER), 'rb'
).read().strip()
......@@ -360,7 +373,7 @@ class FilesystemHelper:
def isSecure(self, path):
"""Ensure that (POSIX) path mode bits are 0700."""
return (os.stat(path).st_mode & 077) == 0
return (os.stat(path).st_mode & 0o77) == 0
def checkSecure(self):
if not self.isSecure(self.base_dir):
......@@ -385,7 +398,7 @@ class FilesystemHelper:
if create and not os.path.exists(path):
try:
os.makedirs(path, 0700)
os.makedirs(path, 0o700)
except OSError:
# We might have lost a race. If so, the directory
# must exist now
......@@ -544,8 +557,8 @@ class BushyLayout(object):
directories = []
# Create the bushy directory structure with the least significant byte
# first
for byte in str(oid):
directories.append('0x%s' % binascii.hexlify(byte))
for byte in oid.decode():
directories.append('0x%s' % binascii.hexlify(byte.encode()))
return os.path.sep.join(directories)
def path_to_oid(self, path):
......@@ -927,8 +940,8 @@ def is_blob_record(record):
storage to another.
"""
if record and ('ZODB.blob' in record):
unpickler = cPickle.Unpickler(cStringIO.StringIO(record))
if record and (b'ZODB.blob' in record):
unpickler = serialize._Unpickler(BytesIO(record))
unpickler.find_global = find_global_Blob
try:
......
......@@ -82,7 +82,11 @@ class Broken(object):
>>> r[2]
{'x': 1}
>>> import cPickle
>>> try:
... import cPickle
... except ImportError:
... # Py3
... import pickle as cPickle
>>> a2 = cPickle.loads(cPickle.dumps(a, 1))
>>> a2
<broken not.there.Atall instance>
......
......@@ -12,14 +12,17 @@
#
##############################################################################
"""Open database and storage from a configuration."""
import os
from cStringIO import StringIO
import ZConfig
import ZODB
try:
from cStringIO import StringIO
except ImportError:
# Py3
from io import StringIO
db_schema_path = os.path.join(ZODB.__path__[0], "config.xml")
_db_schema = None
......
......@@ -31,7 +31,7 @@
# Because
# - the mapping from suffix to data contains at most 65535 entries,
# - this is an in-memory data structure
# - new keys are inserted sequentially,
# - new keys are inserted sequentially,
# we use a BTree bucket instead of a full BTree to store the results.
#
# We use p64 to convert integers to 8-byte strings and lop off the two
......@@ -41,11 +41,18 @@
from __future__ import with_statement
import cPickle
import struct
from BTrees._fsBTree import fsBucket
from BTrees.OOBTree import OOBTree
import six
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
# convert between numbers and six-byte strings
......@@ -53,7 +60,7 @@ def num2str(n):
return struct.pack(">Q", n)[2:]
def str2num(s):
return struct.unpack(">Q", "\000\000" + s)[0]
return struct.unpack(">Q", b"\000\000" + s)[0]
def prefix_plus_one(s):
num = str2num(s)
......@@ -74,7 +81,7 @@ class fsIndex(object):
return dict(
state_version = 1,
_data = [(k, v.toString())
for (k, v) in self._data.iteritems()
for (k, v) in six.iteritems(self._data)
]
)
......@@ -97,19 +104,19 @@ class fsIndex(object):
def save(self, pos, fname):
with open(fname, 'wb') as f:
pickler = cPickle.Pickler(f, 1)
pickler = pickle.Pickler(f, 1)
pickler.fast = True
pickler.dump(pos)
for k, v in self._data.iteritems():
for k, v in six.iteritems(self._data):
pickler.dump((k, v.toString()))
pickler.dump(None)
@classmethod
def load(class_, fname):
with open(fname, 'rb') as f:
unpickler = cPickle.Unpickler(f)
unpickler = pickle.Unpickler(f)
pos = unpickler.load()
if not isinstance(pos, (int, long)):
if not isinstance(pos, int):
return pos # Old format
index = class_()
data = index._data
......@@ -143,14 +150,14 @@ class fsIndex(object):
treekey = key[:6]
tree = self._data.get(treekey)
if tree is None:
raise KeyError, key
raise KeyError(key)
del tree[key[6:]]
if not tree:
del self._data[treekey]
def __len__(self):
r = 0
for tree in self._data.itervalues():
for tree in six.itervalues(self._data):
r += len(tree)
return r
......@@ -175,30 +182,30 @@ class fsIndex(object):
self._data.clear()
def __iter__(self):
for prefix, tree in self._data.iteritems():
for prefix, tree in six.iteritems(self._data):
for suffix in tree:
yield prefix + suffix
iterkeys = __iter__
def keys(self):
return list(self.iterkeys())
return list(six.iterkeys(self))
def iteritems(self):
for prefix, tree in self._data.iteritems():
for suffix, value in tree.iteritems():
for prefix, tree in six.iteritems(self._data):
for suffix, value in six.iteritems(tree):
yield (prefix + suffix, str2num(value))
def items(self):
return list(self.iteritems())
def itervalues(self):
for tree in self._data.itervalues():
for value in tree.itervalues():
for tree in six.itervalues(self._data):
for value in six.itervalues(tree):
yield str2num(value)
def values(self):
return list(self.itervalues())
return list(six.itervalues(self))
# Comment below applies for the following minKey and maxKey methods
#
......
......@@ -51,6 +51,7 @@ Options:
Important: The ZODB package must be importable. You may need to adjust
PYTHONPATH accordingly.
"""
from __future__ import print_function
# Algorithm:
#
......@@ -69,7 +70,12 @@ import os
import getopt
import time
from struct import unpack
from cPickle import loads
try:
from cPickle import loads
except ImportError:
# Py3
from pickle import loads
try:
import ZODB
......@@ -81,7 +87,7 @@ except ImportError:
import ZODB
import ZODB.FileStorage
from ZODB.utils import u64
from ZODB.utils import u64, as_text
from ZODB.FileStorage import TransactionRecord
from persistent.TimeStamp import TimeStamp
......@@ -89,9 +95,9 @@ from persistent.TimeStamp import TimeStamp
def die(mess='', show_docstring=False):
if mess:
print >> sys.stderr, mess + '\n'
print(mess + '\n', file=sys.stderr)
if show_docstring:
print >> sys.stderr, __doc__ % sys.argv[0]
print(__doc__ % sys.argv[0], file=sys.stderr)
sys.exit(1)
class ErrorFound(Exception):
......@@ -108,6 +114,7 @@ def read_txn_header(f, pos, file_size, outp, ltid):
raise EOFError
tid, stl, status, ul, dl, el = unpack(">8s8scHHH",h)
status = as_text(status)
tl = u64(stl)
if pos + (tl + 8) > file_size:
......@@ -224,9 +231,9 @@ def scan(f, pos):
def iprogress(i):
if i % 2:
print ".",
print(".", end=' ')
else:
print (i/2) % 10,
print((i/2) % 10, end=' ')
sys.stdout.flush()
def progress(p):
......@@ -236,7 +243,7 @@ def progress(p):
def main():
try:
opts, args = getopt.getopt(sys.argv[1:], "fv:pP:")
except getopt.error, msg:
except getopt.error as msg:
die(str(msg), show_docstring=True)
if len(args) != 2:
......@@ -259,7 +266,7 @@ def main():
recover(inp, outp, verbose, partial, force, pack)
def recover(inp, outp, verbose=0, partial=False, force=False, pack=None):
print "Recovering", inp, "into", outp
print("Recovering", inp, "into", outp)
if os.path.exists(outp) and not force:
die("%s exists" % outp)
......@@ -277,7 +284,7 @@ def recover(inp, outp, verbose=0, partial=False, force=False, pack=None):
prog1 = 0
undone = 0
pos = 4L
pos = 4
ltid = None
while pos:
try:
......@@ -286,13 +293,13 @@ def recover(inp, outp, verbose=0, partial=False, force=False, pack=None):
break
except (KeyboardInterrupt, SystemExit):
raise
except Exception, err:
print "error reading txn header:", err
except Exception as err:
print("error reading txn header:", err)
if not verbose:
progress(prog1)
pos = scan(f, pos)
if verbose > 1:
print "looking for valid txn header at", pos
print("looking for valid txn header at", pos)
continue
ltid = tid
......@@ -311,22 +318,22 @@ def recover(inp, outp, verbose=0, partial=False, force=False, pack=None):
t = TimeStamp(tid)
if t <= _ts:
if ok:
print ("Time stamps out of order %s, %s" % (_ts, t))
print(("Time stamps out of order %s, %s" % (_ts, t)))
ok = 0
_ts = t.laterThan(_ts)
tid = _ts.raw()
else:
_ts = t
if not ok:
print ("Time stamps back in order %s" % (t))
print(("Time stamps back in order %s" % (t)))
ok = 1
ofs.tpc_begin(txn, tid, txn.status)
if verbose:
print "begin", pos, _ts,
print("begin", pos, _ts, end=' ')
if verbose > 1:
print
print()
sys.stdout.flush()
nrec = 0
......@@ -338,36 +345,36 @@ def recover(inp, outp, verbose=0, partial=False, force=False, pack=None):
else:
l = len(r.data)
print "%7d %s %s" % (u64(r.oid), l)
print("%7d %s %s" % (u64(r.oid), l))
ofs.restore(r.oid, r.tid, r.data, '', r.data_txn,
txn)
nrec += 1
except (KeyboardInterrupt, SystemExit):
raise
except Exception, err:
except Exception as err:
if partial and nrec:
ofs._status = "p"
ofs.tpc_vote(txn)
ofs.tpc_finish(txn)
if verbose:
print "partial"
print("partial")
else:
ofs.tpc_abort(txn)
print "error copying transaction:", err
print("error copying transaction:", err)
if not verbose:
progress(prog1)
pos = scan(f, pos)
if verbose > 1:
print "looking for valid txn header at", pos
print("looking for valid txn header at", pos)
else:
ofs.tpc_vote(txn)
ofs.tpc_finish(txn)
if verbose:
print "finish"
print("finish")
sys.stdout.flush()
if not verbose:
prog = pos * 20l / file_size
prog = pos * 20 / file_size
while prog > prog1:
prog1 = prog1 + 1
iprogress(prog1)
......@@ -375,12 +382,12 @@ def recover(inp, outp, verbose=0, partial=False, force=False, pack=None):
bad = file_size - undone - ofs._pos
print "\n%s bytes removed during recovery" % bad
print("\n%s bytes removed during recovery" % bad)
if undone:
print "%s bytes of undone transaction data were skipped" % undone
print("%s bytes of undone transaction data were skipped" % undone)
if pack is not None:
print "Packing ..."
print("Packing ...")
from ZODB.serialize import referencesf
ofs.pack(pack, referencesf)
......
......@@ -18,7 +18,6 @@ TODO: This module needs tests.
Caution: This file needs to be kept in sync with FileStorage.py.
"""
import cPickle
import struct
from ZODB.FileStorage.format import TRANS_HDR, DATA_HDR, TRANS_HDR_LEN
......@@ -26,6 +25,13 @@ from ZODB.FileStorage.format import DATA_HDR_LEN
from ZODB.utils import u64
from persistent.TimeStamp import TimeStamp
try:
import cPickle
except ImportError:
# Py3
import pickle as cPickle
class TxnHeader:
"""Object representing a transaction record header.
......
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
#!/usr/bin/env python2.4
# Based on a transaction analyzer by Matt Kromer.
......@@ -47,39 +60,39 @@ def shorten(s, n):
return "..." + s
def report(rep):
print "Processed %d records in %d transactions" % (rep.OIDS, rep.TIDS)
print "Average record size is %7.2f bytes" % (rep.DBYTES * 1.0 / rep.OIDS)
print ("Average transaction size is %7.2f bytes" %
(rep.DBYTES * 1.0 / rep.TIDS))
print("Processed %d records in %d transactions" % (rep.OIDS, rep.TIDS))
print("Average record size is %7.2f bytes" % (rep.DBYTES * 1.0 / rep.OIDS))
print(("Average transaction size is %7.2f bytes" %
(rep.DBYTES * 1.0 / rep.TIDS)))
print "Types used:"
print("Types used:")
fmt = "%-46s %7s %9s %6s %7s"
fmtp = "%-46s %7d %9d %5.1f%% %7.2f" # per-class format
fmts = "%46s %7d %8dk %5.1f%% %7.2f" # summary format
print fmt % ("Class Name", "Count", "TBytes", "Pct", "AvgSize")
print fmt % ('-'*46, '-'*7, '-'*9, '-'*5, '-'*7)
print(fmt % ("Class Name", "Count", "TBytes", "Pct", "AvgSize"))
print(fmt % ('-'*46, '-'*7, '-'*9, '-'*5, '-'*7))
typemap = rep.TYPEMAP.keys()
typemap.sort()
cumpct = 0.0
for t in typemap:
pct = rep.TYPESIZE[t] * 100.0 / rep.DBYTES
cumpct += pct
print fmtp % (shorten(t, 46), rep.TYPEMAP[t], rep.TYPESIZE[t],
pct, rep.TYPESIZE[t] * 1.0 / rep.TYPEMAP[t])
print(fmtp % (shorten(t, 46), rep.TYPEMAP[t], rep.TYPESIZE[t],
pct, rep.TYPESIZE[t] * 1.0 / rep.TYPEMAP[t]))
print fmt % ('='*46, '='*7, '='*9, '='*5, '='*7)
print "%46s %7d %9s %6s %6.2fk" % ('Total Transactions', rep.TIDS, ' ',
' ', rep.DBYTES * 1.0 / rep.TIDS / 1024.0)
print fmts % ('Total Records', rep.OIDS, rep.DBYTES / 1024.0, cumpct,
rep.DBYTES * 1.0 / rep.OIDS)
print(fmt % ('='*46, '='*7, '='*9, '='*5, '='*7))
print("%46s %7d %9s %6s %6.2fk" % ('Total Transactions', rep.TIDS, ' ',
' ', rep.DBYTES * 1.0 / rep.TIDS / 1024.0))
print(fmts % ('Total Records', rep.OIDS, rep.DBYTES / 1024.0, cumpct,
rep.DBYTES * 1.0 / rep.OIDS))
print fmts % ('Current Objects', rep.COIDS, rep.CBYTES / 1024.0,
print(fmts % ('Current Objects', rep.COIDS, rep.CBYTES / 1024.0,
rep.CBYTES * 100.0 / rep.DBYTES,
rep.CBYTES * 1.0 / rep.COIDS)
rep.CBYTES * 1.0 / rep.COIDS))
if rep.FOIDS:
print fmts % ('Old Objects', rep.FOIDS, rep.FBYTES / 1024.0,
print(fmts % ('Old Objects', rep.FOIDS, rep.FBYTES / 1024.0,
rep.FBYTES * 100.0 / rep.DBYTES,
rep.FBYTES * 1.0 / rep.FOIDS)
rep.FBYTES * 1.0 / rep.FOIDS))
def analyze(path):
fs = FileStorage(path, read_only=1)
......@@ -97,7 +110,7 @@ def analyze_trans(report, txn):
def get_type(record):
try:
unpickled = FakeUnpickler(StringIO(record.data)).load()
except FakeError, err:
except FakeError as err:
return "%s.%s" % (err.module, err.name)
except:
raise
......@@ -133,8 +146,8 @@ def analyze_rec(report, record):
report.CBYTES += size - fsize
report.TYPEMAP[type] = report.TYPEMAP.get(type, 0) + 1
report.TYPESIZE[type] = report.TYPESIZE.get(type, 0) + size
except Exception, err:
print err
except Exception as err:
print(err)
if __name__ == "__main__":
path = sys.argv[1]
......
......@@ -7,6 +7,16 @@ usage: checkbtrees.py data.fs
Try to find all the BTrees in a Data.fs, call their _check() methods,
and run them through BTrees.check.check().
"""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
import ZODB
from ZODB.FileStorage import FileStorage
......@@ -26,7 +36,7 @@ def add_if_new_persistent(L, obj, path):
getattr(obj, '_', None) # unghostify
if hasattr(obj, '_p_oid'):
oid = obj._p_oid
if not oids_seen.has_key(oid):
if oid not in oids_seen:
L.append((obj, path))
oids_seen[oid] = 1
......@@ -69,7 +79,7 @@ def main(fname=None):
try:
fname, = sys.argv[1:]
except:
print __doc__
print(__doc__)
sys.exit(2)
fs = FileStorage(fname, read_only=1)
......@@ -83,26 +93,26 @@ def main(fname=None):
obj, path = todo.pop(0)
found += 1
if not path:
print "<root>", repr(obj)
print("<root>", repr(obj))
else:
print path, repr(obj)
print(path, repr(obj))
mod = str(obj.__class__.__module__)
if mod.startswith("BTrees"):
if hasattr(obj, "_check"):
try:
obj._check()
except AssertionError, msg:
print "*" * 60
print msg
print "*" * 60
except AssertionError as msg:
print("*" * 60)
print(msg)
print("*" * 60)
try:
check(obj)
except AssertionError, msg:
print "*" * 60
print msg
print "*" * 60
except AssertionError as msg:
print("*" * 60)
print(msg)
print("*" * 60)
if found % 100 == 0:
cn.cacheMinimize()
......@@ -115,7 +125,7 @@ def main(fname=None):
newpath = "%s.%s" % (path, k)
add_if_new_persistent(todo, v, newpath)
print "total", len(fs._index), "found", found
print("total", len(fs._index), "found", found)
if __name__ == "__main__":
main()
......@@ -37,13 +37,14 @@ is not recommended (spurious error messages may result).
See testfsoids.py for a tutorial doctest.
"""
from __future__ import print_function
import sys
from ZODB.FileStorage.fsoids import Tracer
def usage():
print __doc__
print(__doc__)
def main():
import getopt
......
......@@ -62,6 +62,10 @@ of objects, it does not attempt to load objects in versions, or non-current
revisions of objects; therefore fsrefs cannot find problems in versions or
in non-current revisions.
"""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
import traceback
......@@ -83,16 +87,16 @@ def report(oid, data, serial, missing):
else:
plural = ""
ts = TimeStamp(serial)
print "oid %s %s.%s" % (hex(u64(oid)), from_mod, from_class)
print "last updated: %s, tid=%s" % (ts, hex(u64(serial)))
print "refers to invalid object%s:" % plural
print("oid %s %s.%s" % (hex(u64(oid)), from_mod, from_class))
print("last updated: %s, tid=%s" % (ts, hex(u64(serial))))
print("refers to invalid object%s:" % plural)
for oid, info, reason in missing:
if isinstance(info, tuple):
description = "%s.%s" % info
else:
description = str(info)
print "\toid %s %s: %r" % (oid_repr(oid), reason, description)
print
print("\toid %s %s: %r" % (oid_repr(oid), reason, description))
print()
def main(path=None):
if path is None:
......
#!/usr/bin/env python2.3
"""Print details statistics from fsdump output."""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
import re
import sys
import six
from six.moves import filter
rx_txn = re.compile("tid=([0-9a-f]+).*size=(\d+)")
rx_data = re.compile("oid=([0-9a-f]+) class=(\S+) size=(\d+)")
......@@ -21,10 +36,10 @@ class Histogram(dict):
self[size] = self.get(size, 0) + 1
def size(self):
return sum(self.itervalues())
return sum(six.itervalues(self))
def mean(self):
product = sum([k * v for k, v in self.iteritems()])
product = sum([k * v for k, v in six.iteritems(self)])
return product / self.size()
def median(self):
......@@ -42,17 +57,17 @@ class Histogram(dict):
def mode(self):
mode = 0
value = 0
for k, v in self.iteritems():
for k, v in six.iteritems(self):
if v > value:
value = v
mode = k
return mode
def make_bins(self, binsize):
maxkey = max(self.iterkeys())
maxkey = max(six.iterkeys(self))
self.binsize = binsize
self.bins = [0] * (1 + maxkey / binsize)
for k, v in self.iteritems():
for k, v in six.iteritems(self):
b = k / binsize
self.bins[b] += v
......@@ -67,41 +82,41 @@ class Histogram(dict):
# Print up to 40 dots for a value
dot = max(maxval / 40, 1)
tot = sum(self.bins)
print name
print "Total", tot,
print "Median", self.median(),
print "Mean", self.mean(),
print "Mode", self.mode(),
print "Max", max(self)
print "One * represents", dot
print(name)
print("Total", tot, end=' ')
print("Median", self.median(), end=' ')
print("Mean", self.mean(), end=' ')
print("Mode", self.mode(), end=' ')
print("Max", max(self))
print("One * represents", dot)
gap = False
cum = 0
for i, n in enumerate(self.bins):
if gaps and (not n or (skip and not n / dot)):
if not gap:
print " ..."
print(" ...")
gap = True
continue
gap = False
p = 100 * n / tot
cum += n
pc = 100 * cum / tot
print "%6d %6d %3d%% %3d%% %s" % (
i * binsize, n, p, pc, "*" * (n / dot))
print
print("%6d %6d %3d%% %3d%% %s" % (
i * binsize, n, p, pc, "*" * (n / dot)))
print()
def class_detail(class_size):
# summary of classes
fmt = "%5s %6s %6s %6s %-50.50s"
labels = ["num", "median", "mean", "mode", "class"]
print fmt % tuple(labels)
print fmt % tuple(["-" * len(s) for s in labels])
for klass, h in sort_byhsize(class_size.iteritems()):
print fmt % (h.size(), h.median(), h.mean(), h.mode(), klass)
print
print(fmt % tuple(labels))
print(fmt % tuple(["-" * len(s) for s in labels]))
for klass, h in sort_byhsize(six.iteritems(class_size)):
print(fmt % (h.size(), h.median(), h.mean(), h.mode(), klass))
print()
# per class details
for klass, h in sort_byhsize(class_size.iteritems(), reverse=True):
for klass, h in sort_byhsize(six.iteritems(class_size), reverse=True):
h.make_bins(50)
if len(filter(None, h.bins)) == 1:
continue
......@@ -109,7 +124,7 @@ def class_detail(class_size):
def revision_detail(lifetimes, classes):
# Report per-class details for any object modified more than once
for name, oids in classes.iteritems():
for name, oids in six.iteritems(classes):
h = Histogram()
keep = False
for oid in dict.fromkeys(oids, 1):
......@@ -179,9 +194,9 @@ def main(path=None):
txn_bytes.add(size)
f.close()
print "Summary: %d txns, %d objects, %d revisions" % (
txn_objects.size(), len(n_updates), n_updates.size())
print
print("Summary: %d txns, %d objects, %d revisions" % (
txn_objects.size(), len(n_updates), n_updates.size()))
print()
txn_bytes.report("Transaction size (bytes)", binsize=1024)
txn_objects.report("Transaction size (objects)", binsize=10)
......
......@@ -14,6 +14,8 @@
#
##############################################################################
"""Tool to dump the last few transactions from a FileStorage."""
from __future__ import print_function
from __future__ import print_function
from ZODB.fstools import prev_txn
......@@ -35,11 +37,11 @@ def main(path, ntxn):
hash = sha1(th.get_raw_data()).digest()
l = len(str(th.get_timestamp())) + 1
th.read_meta()
print "%s: hash=%s" % (th.get_timestamp(),
binascii.hexlify(hash))
print ("user=%r description=%r length=%d offset=%d"
% (th.user, th.descr, th.length, th.get_data_offset()))
print
print("%s: hash=%s" % (th.get_timestamp(),
binascii.hexlify(hash)))
print(("user=%r description=%r length=%d offset=%d"
% (th.user, th.descr, th.length, th.get_data_offset())))
print()
th = th.prev_txn()
i -= 1
......
......@@ -32,6 +32,8 @@ Note: It does not check the consistency of the object pickles. It is
possible for the damage to occur only in the part of the file that
stores object pickles. Those errors will go undetected.
"""
from __future__ import print_function
from __future__ import print_function
# The implementation is based closely on the read_index() function in
# ZODB.FileStorage. If anything about the FileStorage layout changes,
......@@ -90,7 +92,7 @@ def check(path):
if file.read(4) != packed_version:
raise FormatError("invalid file header")
pos = 4L
pos = 4
tid = b'\000' * 8 # lowest possible tid to start
i = 0
while pos:
......@@ -200,7 +202,7 @@ def check_drec(path, file, pos, tpos, tid):
return pos, oid
def usage():
print __doc__
print(__doc__)
sys.exit(-1)
def main(args=None):
......@@ -221,8 +223,8 @@ def main(args=None):
try:
check(args[0])
except FormatError, msg:
print msg
except FormatError as msg:
print(msg)
sys.exit(-1)
chatter("no errors detected")
......
......@@ -39,7 +39,7 @@ class TestCorruptedFS(ZODB.tests.util.TestCase):
self._file.close()
try:
fstest.check(self._temp)
except FormatError, msg:
except FormatError as msg:
mo = re.search(rx, str(msg))
self.failIf(mo is None, "unexpected error: %s" % msg)
else:
......
......@@ -74,6 +74,25 @@ Positional arguments:
Comma separated list of arguments for the source storage, as key=val
pairs. E.g. "name=full;frequency=3600"
"""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
import re
import sys
......@@ -89,28 +108,20 @@ from ZODB.TimeStamp import TimeStamp
PROGRAM = sys.argv[0]
ZERO = '\0'*8
try:
True, False
except NameError:
True = 1
False = 0
def usage(code, msg=''):
print >> sys.stderr, __doc__ % globals()
print(__doc__ % globals(), file=sys.stderr)
if msg:
print >> sys.stderr, msg
print(msg, file=sys.stderr)
sys.exit(code)
def error(code, msg):
print >> sys.stderr, msg
print "use --help for usage message"
print(msg, file=sys.stderr)
print("use --help for usage message")
sys.exit(code)
def main():
try:
opts, args = getopt.getopt(
......@@ -119,7 +130,7 @@ def main():
['help', 'verbose',
'output=', 'profile', 'storage_types',
'max=', 'skip=', 'dtype=', 'stype=', 'timestamps'])
except getopt.error, msg:
except getopt.error as msg:
error(2, msg)
class Options:
......@@ -188,7 +199,7 @@ def main():
options.outclosep = True
if options.verbose > 0:
print 'Opening source database...'
print('Opening source database...')
modname, sconv = StorageTypes.storage_types[options.stype]
kw = sconv(**srckws)
__import__(modname)
......@@ -196,7 +207,7 @@ def main():
srcdb = sclass(**kw)
if options.verbose > 0:
print 'Opening destination database...'
print('Opening destination database...')
modname, dconv = StorageTypes.storage_types[options.dtype]
kw = dconv(**destkws)
__import__(modname)
......@@ -208,7 +219,7 @@ def main():
doit(srcdb, dstdb, options)
t1 = time.time()
if options.verbose > 0:
print 'Migration time: %8.3f' % (t1-t0)
print('Migration time: %8.3f' % (t1-t0))
finally:
# Done
srcdb.close()
......@@ -217,7 +228,6 @@ def main():
options.outfp.close()
def doit(srcdb, dstdb, options):
outfp = options.outfp
profilep = options.profilep
......@@ -226,7 +236,7 @@ def doit(srcdb, dstdb, options):
largest_pickle = 0
largest_txn_in_size = 0
largest_txn_in_objects = 0
total_pickle_size = 0L
total_pickle_size = 0
total_object_count = 0
# Ripped from BaseStorage.copyTransactionsFrom()
ts = None
......@@ -235,15 +245,15 @@ def doit(srcdb, dstdb, options):
counter = 0
skipper = 0
if options.timestamps:
print "%4s. %26s %6s %8s %5s %5s %5s %5s %5s" % (
print("%4s. %26s %6s %8s %5s %5s %5s %5s %5s" % (
"NUM", "TID AS TIMESTAMP", "OBJS", "BYTES",
# Does anybody know what these times mean?
"t4-t0", "t1-t0", "t2-t1", "t3-t2", "t4-t3")
"t4-t0", "t1-t0", "t2-t1", "t3-t2", "t4-t3"))
else:
print "%4s. %20s %6s %8s %6s %6s %6s %6s %6s" % (
print("%4s. %20s %6s %8s %6s %6s %6s %6s %6s" % (
"NUM", "TRANSACTION ID", "OBJS", "BYTES",
# Does anybody know what these times mean?
"t4-t0", "t1-t0", "t2-t1", "t3-t2", "t4-t3")
"t4-t0", "t1-t0", "t2-t1", "t3-t2", "t4-t3"))
for txn in srcdb.iterator():
skipper += 1
if skipper <= options.skiptxn:
......@@ -258,19 +268,19 @@ def doit(srcdb, dstdb, options):
t = TimeStamp(tid)
if t <= ts:
if ok:
print >> sys.stderr, (
'Time stamps are out of order %s, %s' % (ts, t))
print((
'Time stamps are out of order %s, %s' % (ts, t)), file=sys.stderr)
ok = False
ts = t.laterThan(ts)
tid = ts.raw()
else:
ts = t
if not ok:
print >> sys.stderr, (
'Time stamps are back in order %s' % t)
print((
'Time stamps are back in order %s' % t), file=sys.stderr)
ok = True
if verbose > 1:
print ts
print(ts)
prof = None
if profilep and (counter % 100) == 0:
......@@ -293,7 +303,7 @@ def doit(srcdb, dstdb, options):
vstr = 'norev'
else:
vstr = r.version
print utils.U64(oid), vstr, len(r.data)
print(utils.U64(oid), vstr, len(r.data))
oldrevid = prevrevids.get(oid, ZERO)
result = dstdb.store(oid, oldrevid, r.data, r.version, txn)
newrevids.store(oid, result)
......@@ -320,8 +330,8 @@ def doit(srcdb, dstdb, options):
else:
tidstr = utils.U64(tid)
format = "%4d. %20s %6d %8d %6.4f %6.4f %6.4f %6.4f %6.4f"
print >> outfp, format % (skipper, tidstr, objects, size,
t4-t0, t1-t0, t2-t1, t3-t2, t4-t3)
print(format % (skipper, tidstr, objects, size,
t4-t0, t1-t0, t2-t1, t3-t2, t4-t3), file=outfp)
total_pickle_size += size
total_object_count += objects
......@@ -330,14 +340,13 @@ def doit(srcdb, dstdb, options):
fp = open('profile-%02d.txt' % (counter / 100), 'wb')
marshal.dump(prof.stats, fp)
fp.close()
print >> outfp, "Largest pickle: %8d" % largest_pickle
print >> outfp, "Largest transaction: %8d" % largest_txn_in_size
print >> outfp, "Largest object count: %8d" % largest_txn_in_objects
print >> outfp, "Total pickle size: %14d" % total_pickle_size
print >> outfp, "Total object count: %8d" % total_object_count
print("Largest pickle: %8d" % largest_pickle, file=outfp)
print("Largest transaction: %8d" % largest_txn_in_size, file=outfp)
print("Largest object count: %8d" % largest_txn_in_objects, file=outfp)
print("Total pickle size: %14d" % total_pickle_size, file=outfp)
print("Total object count: %8d" % total_object_count, file=outfp)
# helper to deal with differences between old-style store() return and
# new-style store() return that supports ZEO
......@@ -366,6 +375,5 @@ class RevidAccumulator:
return self.data
if __name__ == '__main__':
main()
......@@ -13,6 +13,8 @@
##############################################################################
"""A script to migrate a blob directory into a different layout.
"""
from __future__ import print_function
from __future__ import print_function
import logging
import optparse
......@@ -41,8 +43,8 @@ def migrate(source, dest, layout):
source_fsh.create()
dest_fsh = FilesystemHelper(dest, layout)
dest_fsh.create()
print "Migrating blob data from `%s` (%s) to `%s` (%s)" % (
source, source_fsh.layout_name, dest, dest_fsh.layout_name)
print("Migrating blob data from `%s` (%s) to `%s` (%s)" % (
source, source_fsh.layout_name, dest, dest_fsh.layout_name))
for oid, path in source_fsh.listOIDs():
dest_path = dest_fsh.getPathForOID(oid, create=True)
files = os.listdir(path)
......@@ -50,7 +52,7 @@ def migrate(source, dest, layout):
source_file = os.path.join(path, file)
dest_file = os.path.join(dest_path, file)
link_or_copy(source_file, dest_file)
print "\tOID: %s - %s files " % (oid_repr(oid), len(files))
print("\tOID: %s - %s files " % (oid_repr(oid), len(files)))
def main(source=None, dest=None, layout="bushy"):
......
......@@ -7,11 +7,17 @@ usage: netspace.py [-P | -v] data.fs
-P: do a pack first
-v: print info for all objects, even if a traversal path isn't found
"""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
import ZODB
from ZODB.FileStorage import FileStorage
from ZODB.utils import U64, get_pickle_metadata
from ZODB.referencesf import referencesf
from six.moves import filter
def find_paths(root, maxdist):
"""Find Python attribute traversal paths for objects to maxdist distance.
......@@ -67,7 +73,7 @@ def main(path):
data, serialno = fs.load(oid, '')
size = len(data)
for suboid in referencesf(data):
if seen.has_key(suboid):
if suboid in seen:
continue
seen[suboid] = 1
size += _total_size(suboid, seen)
......@@ -93,7 +99,7 @@ def main(path):
mod, klass = get_pickle_metadata(data)
refs = referencesf(data)
path = paths.get(oid, '-')
print fmt % (U64(oid), len(data), total_size(oid), path, mod, klass)
print(fmt % (U64(oid), len(data), total_size(oid), path, mod, klass))
def Main():
import sys
......@@ -107,13 +113,13 @@ def Main():
try:
opts, args = getopt.getopt(sys.argv[1:], 'Pv')
path, = args
except getopt.error, err:
print err
print __doc__
except getopt.error as err:
print(err)
print(__doc__)
sys.exit(2)
except ValueError:
print "expected one argument, got", len(args)
print __doc__
print("expected one argument, got", len(args))
print(__doc__)
sys.exit(2)
for o, v in opts:
if o == '-P':
......
......@@ -70,10 +70,11 @@ Options for -R/--recover:
Note: for the stdout case, the index file will **not** be restored
automatically.
"""
from __future__ import print_function
import os
import shutil
import sys
from six.moves import filter
try:
# the hashlib package is available from Python 2.5
from hashlib import md5
......@@ -110,9 +111,9 @@ def usage(code, msg=''):
if code == 0:
outfp = sys.stdout
print >> outfp, __doc__ % globals()
print(__doc__ % globals(), file=outfp)
if msg:
print >> outfp, msg
print(msg, file=outfp)
sys.exit(code)
......@@ -120,7 +121,7 @@ def usage(code, msg=''):
def log(msg, *args):
if VERBOSE:
# Use stderr here so that -v flag works with -R and no -o
print >> sys.stderr, msg % args
print(msg % args, file=sys.stderr)
def parseargs(argv):
......@@ -140,7 +141,7 @@ def parseargs(argv):
'date=',
'output=',
])
except getopt.error, msg:
except getopt.error as msg:
usage(1, msg)
class Options:
......@@ -229,7 +230,7 @@ def fsync(afile):
# passed in all to func(). Leaves the file position just after the
# last byte read.
def dofile(func, fp, n=None):
bytesread = 0L
bytesread = 0
while n is None or n > 0:
if n is None:
todo = READCHUNK
......@@ -336,9 +337,9 @@ def find_files(options):
if not when:
when = gen_filename(options, ext='')
log('looking for files between last full backup and %s...', when)
all = filter(is_data_file, os.listdir(options.repository))
all.sort()
all.reverse() # newest file first
# newest file first
all = sorted(
filter(is_data_file, os.listdir(options.repository)), reverse=True)
# Find the last full backup before date, then include all the
# incrementals between that full backup and "when".
needed = []
......@@ -376,8 +377,8 @@ def scandat(repofiles):
fn = startpos = endpos = sum = None # assume .dat file missing or empty
try:
fp = open(datfile)
except IOError, e:
if e.errno <> errno.ENOENT:
except IOError as e:
if e.errno != errno.ENOENT:
raise
else:
# We only care about the last one.
......@@ -385,15 +386,14 @@ def scandat(repofiles):
fp.close()
if lines:
fn, startpos, endpos, sum = lines[-1].split()
startpos = long(startpos)
endpos = long(endpos)
startpos = int(startpos)
endpos = int(endpos)
return fn, startpos, endpos, sum
def delete_old_backups(options):
# Delete all full backup files except for the most recent full backup file
all = filter(is_data_file, os.listdir(options.repository))
all.sort()
all = sorted(filter(is_data_file, os.listdir(options.repository)))
deletable = []
full = []
......@@ -456,7 +456,7 @@ def do_full_backup(options):
# Write the data file for this full backup
datfile = os.path.splitext(dest)[0] + '.dat'
fp = open(datfile, 'w')
print >> fp, dest, 0, pos, sum
print(dest, 0, pos, sum, file=fp)
fp.flush()
os.fsync(fp.fileno())
fp.close()
......@@ -492,7 +492,7 @@ def do_incremental_backup(options, reposz, repofiles):
datfile = os.path.splitext(fullfile)[0] + '.dat'
# This .dat file better exist. Let the exception percolate if not.
fp = open(datfile, 'a')
print >> fp, dest, reposz, pos, sum
print(dest, reposz, pos, sum, file=fp)
fp.flush()
os.fsync(fp.fileno())
fp.close()
......@@ -593,7 +593,7 @@ def do_recover(options):
log('Recovering file to %s', options.output)
outfp = open(options.output, 'wb')
reposz, reposum = concat(repofiles, outfp)
if outfp <> sys.stdout:
if outfp != sys.stdout:
outfp.close()
log('Recovered %s bytes, md5: %s', reposz, reposum)
......@@ -615,15 +615,15 @@ def main(argv=None):
if options.mode == BACKUP:
try:
do_backup(options)
except WouldOverwriteFiles, e:
print >> sys.stderr, str(e)
except WouldOverwriteFiles as e:
print(str(e), file=sys.stderr)
sys.exit(1)
else:
assert options.mode == RECOVER
try:
do_recover(options)
except NoFiles, e:
print >> sys.stderr, str(e)
except NoFiles as e:
print(str(e), file=sys.stderr)
sys.exit(1)
......
......@@ -8,15 +8,22 @@ The current implementation only supports FileStorage.
Current limitations / simplifications: Ignores revisions and versions.
"""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from ZODB.FileStorage import FileStorage
from ZODB.utils import U64, get_pickle_metadata
import six
def run(path, v=0):
fs = FileStorage(path, read_only=1)
# break into the file implementation
if hasattr(fs._index, 'iterkeys'):
iter = fs._index.iterkeys()
iter = six.iterkeys(fs._index)
else:
iter = fs._index.keys()
totals = {}
......@@ -29,25 +36,25 @@ def run(path, v=0):
count += 1
totals[key] = bytes, count
if v:
print "%8s %5d %s" % (U64(oid), len(data), key)
print("%8s %5d %s" % (U64(oid), len(data), key))
L = totals.items()
L.sort(lambda a, b: cmp(a[1], b[1]))
L.reverse()
print "Totals per object class:"
print("Totals per object class:")
for key, (bytes, count) in L:
print "%8d %8d %s" % (count, bytes, key)
print("%8d %8d %s" % (count, bytes, key))
def main():
import sys
import getopt
try:
opts, args = getopt.getopt(sys.argv[1:], "v")
except getopt.error, msg:
print msg
print "usage: space.py [-v] Data.fs"
except getopt.error as msg:
print(msg)
print("usage: space.py [-v] Data.fs")
sys.exit(2)
if len(args) != 1:
print "usage: space.py [-v] Data.fs"
print("usage: space.py [-v] Data.fs")
sys.exit(2)
v = 0
for o, a in opts:
......
This diff is collapsed.
......@@ -115,6 +115,10 @@ Usage: loadmail2 [options]
$Id$
"""
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
from __future__ import print_function
import mailbox
import math
......@@ -125,6 +129,8 @@ import sys
import threading
import time
import transaction
from six.moves import filter
import six
class JobProducer:
......@@ -183,7 +189,7 @@ class MBox:
self.number = start
while min:
mb.next()
six.advance_iterator(mb)
min -= 1
self._lock = threading.Lock()
......@@ -195,7 +201,7 @@ class MBox:
try:
if self._max > 0 and self.number >= self._max:
raise IndexError(self.number + 1)
message = self._mbox.next()
message = six.advance_iterator(self._mbox)
message.body = message.fp.read()
message.headers = list(message.headers)
self.number += 1
......@@ -315,14 +321,14 @@ def run1(tid, db, factory, job, args):
(start, wcomp, ccomp, rconflicts, wconflicts, wcommit, ccommit, r
) = do(db, job, args)
start = "%.4d-%.2d-%.2d %.2d:%.2d:%.2d" % time.localtime(start)[:6]
print "%s %s %8.3g %8.3g %s %s\t%8.3g %8.3g %s %r" % (
print("%s %s %8.3g %8.3g %s %s\t%8.3g %8.3g %s %r" % (
start, tid, wcomp, ccomp, rconflicts, wconflicts, wcommit, ccommit,
factory.__name__, r)
factory.__name__, r))
def run(jobs, tid=b''):
import Zope2
while 1:
factory, job, args, repeatp = jobs.next()
factory, job, args, repeatp = six.advance_iterator(jobs)
run1(tid, Zope2.DB, factory, job, args)
if repeatp:
while 1:
......@@ -386,7 +392,7 @@ class IndexJob:
self.mbox, self.number, self.max = mbox, int(number), int(max)
def create(self):
messages = [self.mbox.next() for i in range(self.number)]
messages = [six.advance_iterator(self.mbox) for i in range(self.number)]
return index, (messages, self.catalog, self.max)
......@@ -433,7 +439,7 @@ def edit(connection, mbox, catalog=1):
m = wordre.search(word)
if m:
word = m.group(1).lower()
if (not wordsd.has_key(word)) and word not in stop:
if (word not in wordsd) and word not in stop:
words.append(word)
wordsd[word] = 1
del text[j]
......@@ -498,7 +504,7 @@ class SearchJob:
self.terms = words
number = min(int(number), len(self.terms))
self.number = range(number)
self.number = list(range(number))
def create(self):
return search, (self.terms, self.number)
......@@ -702,9 +708,9 @@ def collect_options(args, jobs, options):
if name == 'options':
fname = args.pop(0)
d = {}
execfile(fname, d)
exec(compile(open(fname).read(), fname, 'exec'), d)
collect_options(list(d['options']), jobs, options)
elif options.has_key(name):
elif name in options:
v = args.pop(0)
if options[name] != None:
raise ValueError(
......@@ -714,31 +720,31 @@ def collect_options(args, jobs, options):
options[name] = v
elif name == 'setup':
options['setup'] = 1
elif globals().has_key(name.capitalize()+'Job'):
elif name.capitalize()+'Job' in globals():
job = name
kw = {}
while args and args[0].find("=") > 0:
arg = args.pop(0).split('=')
name, v = arg[0], '='.join(arg[1:])
if kw.has_key(name):
if name in kw:
raise ValueError(
"Duplicate parameter %s for job %s"
% (name, job)
)
kw[name]=v
if kw.has_key('frequency'):
if 'frequency' in kw:
frequency = kw['frequency']
del kw['frequency']
else:
frequency = 1
if kw.has_key('sleep'):
if 'sleep' in kw:
sleep = float(kw['sleep'])
del kw['sleep']
else:
sleep = 0.0001
if kw.has_key('repeat'):
if 'repeat' in kw:
repeatp = float(kw['repeat'])
del kw['repeat']
else:
......@@ -767,10 +773,10 @@ def main(args=None):
if args is None:
args = sys.argv[1:]
if not args:
print __doc__
print(__doc__)
sys.exit(0)
print args
print(args)
random.seed(hash(tuple(args))) # always use the same for the given args
options = {"mbox": None, "threads": None}
......@@ -784,7 +790,7 @@ def main(args=None):
# Perform a ZConfig-based Zope initialization:
zetup(os.path.join(lib_python, '..', '..', 'etc', 'zope.conf'))
if options.has_key('setup'):
if 'setup' in options:
setup(lib_python)
else:
import Zope2
......@@ -794,7 +800,7 @@ def main(args=None):
for job, kw, frequency, sleep, repeatp in jobdefs:
Job = globals()[job.capitalize()+'Job']
if getattr(Job, 'needs_mbox', 0):
if not kw.has_key("mbox"):
if "mbox" not in kw:
if not options["mbox"]:
raise ValueError(
"no mailbox (mbox option) file specified")
......@@ -806,7 +812,7 @@ def main(args=None):
jobs.add(Job(**kw), frequency, sleep, repeatp)
if not jobs:
print "No jobs to execute"
print("No jobs to execute")
return
threads = int(options['threads'] or '0')
......
......@@ -114,7 +114,7 @@ oid
of a database name, an object id, and class meta data.
The following legacy format is also supported.
[oid]
A persistent weak reference
......@@ -133,20 +133,44 @@ A number of legacyforms are defined:
"""
import cPickle
import cStringIO
import logging
import sys
from persistent import Persistent
from persistent.wref import WeakRefMarker, WeakRef
from ZODB import broken
from ZODB.broken import Broken
from ZODB.POSException import InvalidObjectReference
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
try:
from cStringIO import StringIO as BytesIO
except ImportError:
# Py3
from io import BytesIO
if sys.version_info[0] < 3:
_Unpickler = pickle.Unpickler
else:
# Py3: Python 3 doesn't allow assignments to find_global,
# instead, find_class can be overridden
class _Unpickler(pickle.Unpickler):
find_global = None
def find_class(self, modulename, name):
if self.find_global is None:
return super(_Unpickler, self).find_class(modulename, name)
return self.find_global(modulename, name)
_oidtypes = str, type(None)
# Py3: Python 3 uses protocol 3 by default, which is not loadable by Python
# 2. If we want this, we can add a condition here for Python 3.
_protocol = 1
# Might to update or redo coptimizations to reflect weakrefs:
# from ZODB.coptimizations import new_persistent_id
......@@ -172,9 +196,12 @@ class ObjectWriter:
_jar = None
def __init__(self, obj=None):
self._file = cStringIO.StringIO()
self._p = cPickle.Pickler(self._file, 1)
self._p.inst_persistent_id = self.persistent_id
self._file = BytesIO()
self._p = pickle.Pickler(self._file, _protocol)
if sys.version_info[0] < 3:
self._p.inst_persistent_id = self.persistent_id
else:
self._p.persistent_id = self.persistent_id
self._stack = []
if obj is not None:
self._stack.append(obj)
......@@ -193,7 +220,7 @@ class ObjectWriter:
... def db(self):
... return self
... databases = {}
>>> jar = DummyJar()
>>> class O:
... _p_jar = jar
......@@ -284,7 +311,7 @@ class ObjectWriter:
# Not persistent, pickle normally
return None
if not (oid is None or isinstance(oid, str)):
if not (oid is None or isinstance(oid, bytes)):
# Deserves a closer look:
# Make sure it's not a descriptor
......@@ -422,7 +449,7 @@ class ObjectWriter:
return self._dump(meta, obj.__getstate__())
def _dump(self, classmeta, state):
# To reuse the existing cStringIO object, we must reset
# To reuse the existing cBytesIO object, we must reset
# the file position to 0 and truncate the file after the
# new pickle is written.
self._file.seek(0)
......@@ -446,13 +473,15 @@ class NewObjectIterator:
def __iter__(self):
return self
def next(self):
def __next__(self):
if self._stack:
elt = self._stack.pop()
return elt
else:
raise StopIteration
next = __next__
class ObjectReader:
def __init__(self, conn=None, cache=None, factory=None):
......@@ -464,8 +493,8 @@ class ObjectReader:
return self._factory(self._conn, module, name)
def _get_unpickler(self, pickle):
file = cStringIO.StringIO(pickle)
unpickler = cPickle.Unpickler(file)
file = BytesIO(pickle)
unpickler = _Unpickler(file)
unpickler.persistent_load = self._persistent_load
factory = self._factory
conn = self._conn
......@@ -482,7 +511,7 @@ class ObjectReader:
def _persistent_load(self, reference):
if isinstance(reference, tuple):
return self.load_persistent(*reference)
elif isinstance(reference, str):
elif isinstance(reference, bytes):
return self.load_oid(reference)
else:
try:
......@@ -504,7 +533,7 @@ class ObjectReader:
if isinstance(klass, tuple):
klass = self._get_class(*klass)
if issubclass(klass, Broken):
if issubclass(klass, broken.Broken):
# We got a broken class. We might need to make it
# PersistentBroken
if not issubclass(klass, broken.PersistentBroken):
......@@ -590,7 +619,7 @@ class ObjectReader:
# Definitely new style direct class reference
args = ()
if issubclass(klass, Broken):
if issubclass(klass, broken.Broken):
# We got a broken class. We might need to make it
# PersistentBroken
if not issubclass(klass, broken.PersistentBroken):
......@@ -603,7 +632,7 @@ class ObjectReader:
try:
unpickler.load() # skip the class metadata
return unpickler.load()
except EOFError, msg:
except EOFError as msg:
log = logging.getLogger("ZODB.serialize")
log.exception("Unpickling error: %r", pickle)
raise
......@@ -624,28 +653,33 @@ def referencesf(p, oids=None):
"""
refs = []
u = cPickle.Unpickler(cStringIO.StringIO(p))
u.persistent_load = refs
u.noload()
u.noload()
u = pickle.Unpickler(BytesIO(p))
if sys.version_info[0] < 3:
u.persistent_load = refs
u.noload()
u.noload()
else:
u.persistent_load = refs.append
u.load()
u.load()
# Now we have a list of referencs. Need to convert to list of
# oids:
if oids is None:
oids = []
for reference in refs:
if isinstance(reference, tuple):
oid = reference[0]
elif isinstance(reference, str):
elif isinstance(reference, bytes):
oid = reference
else:
assert isinstance(reference, list)
continue
oids.append(oid)
return oids
oid_klass_loaders = {
......@@ -659,9 +693,9 @@ def get_refs(a_pickle):
If the reference doesn't contain class information, then the
klass information is None.
"""
refs = []
u = cPickle.Unpickler(cStringIO.StringIO(a_pickle))
u = pickle.Unpickler(BytesIO(a_pickle))
u.persistent_load = refs
u.noload()
u.noload()
......@@ -674,12 +708,12 @@ def get_refs(a_pickle):
for reference in refs:
if isinstance(reference, tuple):
data = reference
elif isinstance(reference, str):
elif isinstance(reference, bytes):
data = reference, None
else:
assert isinstance(reference, list)
continue
result.append(data)
return result
......@@ -31,6 +31,8 @@ import time
import transaction
import zope.interface
import zope.interface.verify
from six.moves import map
from six.moves import zip
ZERO = b'\0'*8
......@@ -50,12 +52,12 @@ class BasicStorage:
self.assertRaises(
POSException.StorageTransactionError,
self._storage.store,
ZERO, ZERO, '', '', transaction.Transaction())
ZERO, ZERO, b'', '', transaction.Transaction())
self.assertRaises(
POSException.StorageTransactionError,
self._storage.store,
ZERO, 1, '2', '', transaction.Transaction())
ZERO, 1, b'2', '', transaction.Transaction())
self.assertRaises(
POSException.StorageTransactionError,
......@@ -223,7 +225,7 @@ class BasicStorage:
oid = b'\0\0\0\0\0\0\0\xf0'
tid = self._dostore(oid)
tid2 = self._dostore(oid, revid=tid)
data = 'cpersistent\nPersistent\nq\x01.N.' # a simple persistent obj
data = b'cpersistent\nPersistent\nq\x01.N.' # a simple persistent obj
#----------------------------------------------------------------------
# stale read
......@@ -235,7 +237,7 @@ class BasicStorage:
b'\0\0\0\0\0\0\0\0', data, '', t)
self._storage.checkCurrentSerialInTransaction(oid, tid, t)
self._storage.tpc_vote(t)
except POSException.ReadConflictError, v:
except POSException.ReadConflictError as v:
self.assert_(v.oid) == oid
self.assert_(v.serials == (tid2, tid))
else:
......@@ -313,7 +315,7 @@ class BasicStorage:
# First, some initial data.
t = transaction.get()
self._storage.tpc_begin(t)
self._storage.store(ZERO, ZERO, 'x', '', t)
self._storage.store(ZERO, ZERO, b'x', '', t)
self._storage.tpc_vote(t)
tids = []
self._storage.tpc_finish(t, lambda tid: tids.append(tid))
......@@ -323,7 +325,7 @@ class BasicStorage:
t = transaction.get()
self._storage.tpc_begin(t)
self._storage.store(ZERO, tids[0], 'y', '', t)
self._storage.store(ZERO, tids[0], b'y', '', t)
self._storage.tpc_vote(t)
to_join = []
......
......@@ -91,7 +91,7 @@ class ConflictResolvingStorage:
revid2 = self._dostoreNP(oid, revid=revid1, data=zodb_pickle(obj))
try:
self._dostoreNP(oid, revid=revid1, data=zodb_pickle(obj))
except ConflictError, err:
except ConflictError as err:
self.assert_("PCounter2" in str(err))
else:
self.fail("Expected ConflictError")
......
......@@ -18,7 +18,7 @@ import random
import stat
import ZODB.FileStorage
from StorageTestBase import StorageTestBase
from .StorageTestBase import StorageTestBase
class FileStorageCorruptTests(StorageTestBase):
......
......@@ -24,8 +24,16 @@ from ZODB.utils import U64, p64
from transaction import Transaction
import sys
import itertools
import ZODB.blob
import six
try:
from itertools import izip as zip
except ImportError:
# Py3: zip() already returns an iterable.
pass
class IteratorCompare:
......@@ -33,7 +41,7 @@ class IteratorCompare:
eq = self.assertEqual
oid = self._oid
val = val0
for reciter, revid in itertools.izip(txniter, revids + [None]):
for reciter, revid in zip(txniter, revids + [None]):
eq(reciter.tid, revid)
for rec in reciter:
eq(rec.oid, oid)
......@@ -136,7 +144,7 @@ class IteratorStorage(IteratorCompare):
def checkIterateRecordsRepeatedly(self):
self._dostore()
tinfo = self._storage.iterator().next()
tinfo = six.advance_iterator(self._storage.iterator())
self.assertEquals(1, len(list(tinfo)))
self.assertEquals(1, len(list(tinfo)))
......@@ -144,13 +152,16 @@ class IteratorStorage(IteratorCompare):
self._dostore()
iterator = self._storage.iterator()
# We have one transaction with 1 modified object.
txn_1 = iterator.next()
txn_1 = six.advance_iterator(iterator)
self.assertEquals(1, len(list(txn_1)))
# We store another transaction with 1 object, the already running
# iterator does not pick this up.
self._dostore()
self.assertRaises(StopIteration, iterator.next)
if sys.version_info[0] < 3:
self.assertRaises(StopIteration, iterator.next)
else:
self.assertRaises(StopIteration, iterator.__next__)
class ExtendedIteratorStorage(IteratorCompare):
......@@ -176,14 +187,14 @@ class ExtendedIteratorStorage(IteratorCompare):
txniter = self._storage.iterator(revid2, revid3)
self.iter_verify(txniter, [revid2, revid3], 12)
# Specify an upper bound somewhere in between values
revid3a = p64((U64(revid3) + U64(revid4)) / 2)
revid3a = p64(int((U64(revid3) + U64(revid4)) / 2))
txniter = self._storage.iterator(revid2, revid3a)
self.iter_verify(txniter, [revid2, revid3], 12)
# Specify a lower bound somewhere in between values.
# revid2 == revid1+1 is very likely on Windows. Adding 1 before
# dividing ensures that "the midpoint" we compute is strictly larger
# than revid1.
revid1a = p64((U64(revid1) + 1 + U64(revid2)) / 2)
revid1a = p64(int((U64(revid1) + 1 + U64(revid2)) / 2))
assert revid1 < revid1a
txniter = self._storage.iterator(revid1a, revid3a)
self.iter_verify(txniter, [revid2, revid3], 12)
......@@ -201,7 +212,7 @@ class IteratorDeepCompare:
eq = self.assertEqual
iter1 = storage1.iterator()
iter2 = storage2.iterator()
for txn1, txn2 in itertools.izip(iter1, iter2):
for txn1, txn2 in zip(iter1, iter2):
eq(txn1.tid, txn2.tid)
eq(txn1.status, txn2.status)
eq(txn1.user, txn2.user)
......@@ -209,7 +220,7 @@ class IteratorDeepCompare:
eq(txn1.extension, txn2.extension)
itxn1 = iter(txn1)
itxn2 = iter(txn2)
for rec1, rec2 in itertools.izip(itxn1, itxn2):
for rec1, rec2 in zip(itxn1, itxn2):
eq(rec1.oid, rec2.oid)
eq(rec1.tid, rec2.tid)
eq(rec1.data, rec2.data)
......@@ -224,7 +235,7 @@ class IteratorDeepCompare:
fn2 = storage2.loadBlob(rec1.oid, rec1.tid)
self.assert_(fn1 != fn2)
eq(open(fn1, 'rb').read(), open(fn2, 'rb').read())
# Make sure there are no more records left in rec1 and rec2,
# meaning they were the same length.
# Additionally, check that we're backwards compatible to the
......
......@@ -4,6 +4,7 @@ import threading
import time
from persistent.mapping import PersistentMapping
import six
import transaction
import ZODB
......@@ -16,8 +17,7 @@ SHORT_DELAY = 0.01
def sort(l):
"Sort a list in place and return it."
l.sort()
return l
return sorted(l)
class TestThread(threading.Thread):
"""Base class for defining threads that run from unittest.
......@@ -43,7 +43,8 @@ class TestThread(threading.Thread):
def join(self, timeout=None):
threading.Thread.join(self, timeout)
if self._exc_info:
raise self._exc_info[0], self._exc_info[1], self._exc_info[2]
raise six.reraise(
self._exc_info[0], self._exc_info[1], self._exc_info[2])
class ZODBClientThread(TestThread):
......@@ -67,7 +68,7 @@ class ZODBClientThread(TestThread):
else:
for i in range(self.commits):
self.commit(d, i)
self.test.assertEqual(sort(d.keys()), range(self.commits))
self.test.assertEqual(sort(d.keys()), list(range(self.commits)))
def commit(self, d, num):
d[num] = time.time()
......@@ -190,7 +191,7 @@ class ExtStorageClientThread(StorageClientThread):
self.check()
def pick_oid(self):
return random.choice(self.oids.keys())
return random.choice(tuple(self.oids))
def do_load(self):
oid = self.pick_oid()
......
......@@ -89,7 +89,7 @@ class MVCCMappingStorage(MappingStorage):
self._data_snapshot[oid] = tid_data[tid], tid
if self._polled_tid:
if not self._transactions.has_key(self._polled_tid):
if self._polled_tid not in self._transactions:
# This connection is so old that we can no longer enumerate
# all the changes.
self._polled_tid = new_tid
......
......@@ -22,5 +22,9 @@ class MinPO(Persistent):
def __cmp__(self, aMinPO):
return cmp(self.value, aMinPO.value)
# Py3: Python 3 does not support cmp() anymore.
def __eq__(self, aMinPO):
return self.value == aMinPO.value
def __repr__(self):
return "MinPO(%s)" % self.value
This diff is collapsed.
......@@ -18,19 +18,29 @@ semantics (which you can override), and it also provides a helper
method _dostore() which performs a complete store transaction for a
single object revision.
"""
from __future__ import print_function
import sys
import time
from cPickle import Pickler, Unpickler
from cStringIO import StringIO
import transaction
from ZODB.utils import u64
from ZODB.tests.MinPO import MinPO
import ZODB.tests.util
ZERO = '\0'*8
try:
from cPickle import Pickler, Unpickler
except ImportError:
# Py3
from pickle import Pickler, Unpickler
try:
from cStringIO import StringIO as BytesIO
except ImportError:
# Py3
from io import BytesIO
ZERO = b'\0'*8
def snooze():
# In Windows, it's possible that two successive time.time() calls return
......@@ -50,9 +60,12 @@ def _persistent_id(obj):
def zodb_pickle(obj):
"""Create a pickle in the format expected by ZODB."""
f = StringIO()
f = BytesIO()
p = Pickler(f, 1)
p.inst_persistent_id = _persistent_id
if sys.version_info[0] < 3:
p.inst_persistent_id = _persistent_id
else:
p.persistent_id = _persistent_id
klass = obj.__class__
assert not hasattr(obj, '__getinitargs__'), "not ready for constructors"
args = None
......@@ -65,7 +78,7 @@ def zodb_pickle(obj):
p.dump((klass, args))
p.dump(state)
return f.getvalue(1)
return f.getvalue()
def persistent_load(pid):
# helper for zodb_unpickle
......@@ -73,7 +86,7 @@ def persistent_load(pid):
def zodb_unpickle(data):
"""Unpickle an object stored using the format expected by ZODB."""
f = StringIO(data)
f = BytesIO(data)
u = Unpickler(f)
u.persistent_load = persistent_load
klass_info = u.load()
......@@ -95,7 +108,7 @@ def zodb_unpickle(data):
try:
klass = ns[klassname]
except KeyError:
print >> sys.stderr, "can't find %s in %r" % (klassname, ns)
print("can't find %s in %r" % (klassname, ns), file=sys.stderr)
inst = klass()
else:
raise ValueError("expected class info: %s" % repr(klass_info))
......@@ -118,13 +131,13 @@ def handle_all_serials(oid, *args):
"""
d = {}
for arg in args:
if isinstance(arg, str):
if isinstance(arg, bytes):
d[oid] = arg
elif arg is None:
pass
else:
for oid, serial in arg:
if not isinstance(serial, str):
if not isinstance(serial, bytes):
raise serial # error from ZEO server
d[oid] = serial
return d
......
......@@ -82,11 +82,11 @@ class SynchronizedStorage:
def checkStoreNotCommitting(self):
self.verifyNotCommitting(self._storage.store,
OID, SERIALNO, "", "", Transaction())
OID, SERIALNO, b"", "", Transaction())
def checkStoreWrongTrans(self):
self.verifyWrongTrans(self._storage.store,
OID, SERIALNO, "", "", Transaction())
OID, SERIALNO, b"", "", Transaction())
def checkAbortNotCommitting(self):
self._storage.tpc_abort(Transaction())
......
......@@ -29,6 +29,9 @@ from ZODB import DB
from ZODB.tests.MinPO import MinPO
from ZODB.tests.StorageTestBase import zodb_pickle, zodb_unpickle
from six.moves import map
import six
from six.moves import zip
ZERO = '\0'*8
......@@ -677,7 +680,7 @@ class TransactionalUndoStorage:
eq = self.assertEqual
for i in range(BATCHES):
txn = transactions.next()
txn = six.advance_iterator(transactions)
tid = p64(i + 1)
eq(txn.tid, tid)
......@@ -689,11 +692,11 @@ class TransactionalUndoStorage:
eq(L1, L2)
for i in range(BATCHES * OBJECTS):
txn = transactions.next()
txn = six.advance_iterator(transactions)
eq(len([rec for rec in txn if rec.data_txn is None]), 1)
for i in range(BATCHES):
txn = transactions.next()
txn = six.advance_iterator(transactions)
# The undos are performed in reverse order.
otid = p64(BATCHES - i)
......
......@@ -13,6 +13,7 @@
##############################################################################
"""Functional test to produce a dangling reference."""
from __future__ import print_function
import time
......@@ -50,7 +51,7 @@ def create_dangling_ref(db):
# an oid, so a new commit of it won't create a new object.
db.pack()
print repr(c._p_oid)
print(repr(c._p_oid))
o2.child = c
transaction.get().note("set child on o2")
transaction.commit()
......
......@@ -14,6 +14,7 @@
import ZODB.blob
import ZODB.interfaces
import zope.interface
from binascii import hexlify, unhexlify
@zope.interface.implementer(ZODB.interfaces.IStorageWrapper)
class HexStorage(object):
......@@ -46,22 +47,22 @@ class HexStorage(object):
def load(self, oid, version=''):
data, serial = self.base.load(oid, version)
return data[2:].decode('hex'), serial
return unhexlify(data[2:]), serial
def loadBefore(self, oid, tid):
r = self.base.loadBefore(oid, tid)
if r is not None:
data, serial, after = r
return data[2:].decode('hex'), serial, after
return unhexlify(data[2:]), serial, after
else:
return r
def loadSerial(self, oid, serial):
return self.base.loadSerial(oid, serial)[2:].decode('hex')
return unhexlify(self.base.loadSerial(oid, serial)[2:])
def pack(self, pack_time, referencesf, gc=True):
def refs(p, oids=None):
return referencesf(p[2:].decode('hex'), oids)
return referencesf(unhexlify(p[2:]), oids)
return self.base.pack(pack_time, refs, gc)
def registerDB(self, db):
......@@ -73,11 +74,11 @@ class HexStorage(object):
def store(self, oid, serial, data, version, transaction):
return self.base.store(
oid, serial, '.h'+data.encode('hex'), version, transaction)
oid, serial, b'.h'+hexlify(data), version, transaction)
def restore(self, oid, serial, data, version, prev_txn, transaction):
return self.base.restore(
oid, serial, data and ('.h'+data.encode('hex')), version, prev_txn,
oid, serial, data and (b'.h'+hexlify(data)), version, prev_txn,
transaction)
def iterator(self, start=None, stop=None):
......@@ -86,13 +87,13 @@ class HexStorage(object):
def storeBlob(self, oid, oldserial, data, blobfilename, version,
transaction):
return self.base.storeBlob(oid, oldserial, '.h'+data.encode('hex'),
return self.base.storeBlob(oid, oldserial, b'.h'+hexlify(data),
blobfilename, version, transaction)
def restoreBlob(self, oid, serial, data, blobfilename, prev_txn,
transaction):
return self.base.restoreBlob(oid, serial,
data and ('.h'+data.encode('hex')),
data and (b'.h'+hexlify(data)),
blobfilename, prev_txn, transaction)
def invalidateCache(self):
......@@ -102,17 +103,17 @@ class HexStorage(object):
return self.db.invalidate(transaction_id, oids, version)
def references(self, record, oids=None):
return self.db.references(record[2:].decode('hex'), oids)
return self.db.references(unhexlify(record[2:]), oids)
def transform_record_data(self, data):
return '.h'+self._db_transform(data).encode('hex')
return b'.h'+hexlify(self._db_transform(data))
def untransform_record_data(self, data):
return self._db_untransform(data[2:].decode('hex'))
return self._db_untransform(unhexlify(data[2:]))
def record_iternext(self, next=None):
oid, tid, data, next = self.base.record_iternext(next)
return oid, tid, data[2:].decode('hex'), next
return oid, tid, unhexlify(data[2:]), next
def copyTransactionsFrom(self, other):
ZODB.blob.copyTransactionsFromTo(other, self)
......
......@@ -29,9 +29,10 @@ Then, any log output is collected in the handler:
>>> logging.getLogger('foo.bar').exception('eek')
>>> logging.getLogger('foo.bar').info('blah blah')
>>> import six
>>> for record in handler.records:
... print record.name, record.levelname
... print ' ', record.getMessage()
... six.print_(record.name, record.levelname)
... six.print_(' ', record.getMessage())
foo.bar ERROR
eek
foo.bar INFO
......@@ -39,7 +40,7 @@ Then, any log output is collected in the handler:
A similar effect can be gotten by just printing the handler:
>>> print handler
>>> print(handler)
foo.bar ERROR
eek
foo.bar INFO
......
from __future__ import print_function
##############################################################################
#
# Copyright (c) 2001, 2002 Zope Foundation and Contributors.
......@@ -89,7 +90,7 @@ def main(args):
transaction.begin()
rt=jar.root()
key='s%s' % r
if rt.has_key(key): p=rt[key]
if key in rt: p=rt[key]
else: rt[key]=p=P()
for i in range(r):
if z is not None: d=z(data)
......@@ -109,7 +110,7 @@ def main(args):
time.sleep(3)
jar.cacheMinimize(3)
if detailed: print '-'*24
if detailed: print('-'*24)
for r in 1, 10, 100, 1000:
t=results[r]/nrep
sys.stderr.write("mean:\t%s\t%.4f\t%.4f (s/o)\n" % (r, t, t/r))
......
......@@ -96,7 +96,7 @@ class DBMethods(CacheTestBase):
def checkCacheDetail(self):
for name, count in self.db.cacheDetail():
self.assert_(isinstance(name, str))
self.assert_(isinstance(name, bytes))
self.assert_(isinstance(count, int))
def checkCacheExtremeDetail(self):
......@@ -415,7 +415,7 @@ class CacheErrors(unittest.TestCase):
obj2._p_jar = self.jar
try:
self.cache[key] = obj2
except ValueError, detail:
except ValueError as detail:
self.assertEqual(str(detail),
"A different object already has the same oid")
else:
......
......@@ -12,18 +12,34 @@
#
##############################################################################
"""Unit tests for the Connection class."""
from __future__ import with_statement
from __future__ import print_function, with_statement
import doctest
import re
import six
import unittest
from persistent import Persistent
import transaction
import ZODB.tests.util
from ZODB.config import databaseFromString
from ZODB.utils import p64
from persistent import Persistent
from zope.interface.verify import verifyObject
import ZODB.tests.util
from zope.testing import renormalizing
checker = renormalizing.RENormalizing([
# Python 3 bytes add a "b".
(re.compile("b('.*?')"), r"\1"),
# Python 3 removes empty list representation.
(re.compile("set\(\[\]\)"), r"set()"),
# Python 3 adds module name to exceptions.
(re.compile("ZODB.POSException.POSKeyError"), r"POSKeyError"),
(re.compile("ZODB.POSException.ReadConflictError"), r"ReadConflictError"),
(re.compile("ZODB.POSException.ConflictError"), r"ConflictError"),
(re.compile("ZODB.POSException.ConnectionStateError"),
r"ConnectionStateError"),
])
class ConnectionDotAdd(ZODB.tests.util.TestCase):
......@@ -327,30 +343,30 @@ class UserMethodTests(unittest.TestCase):
def test_close_dispatches_to_activity_monitors(self):
r"""doctest that connection close updates activity monitors
Set up a multi-database:
>>> db1 = ZODB.DB(None)
>>> db2 = ZODB.DB(None, databases=db1.databases, database_name='2',
... cache_size=10)
>>> conn1 = db1.open()
>>> conn2 = conn1.get_connection('2')
Add activity monitors to both dbs:
>>> from ZODB.ActivityMonitor import ActivityMonitor
>>> db1.setActivityMonitor(ActivityMonitor())
>>> db2.setActivityMonitor(ActivityMonitor())
Commit a transaction that affects both connections:
>>> conn1.root()[0] = conn1.root().__class__()
>>> conn2.root()[0] = conn2.root().__class__()
>>> transaction.commit()
After closing the primary connection, both monitors should be up to
date:
>>> conn1.close()
>>> len(db1.getActivityMonitor().log)
1
......@@ -433,7 +449,7 @@ def test_transaction_retry_convenience():
>>> for attempt in transaction.manager.attempts():
... with attempt as t:
... t.note('test')
... print dm['ntry'], ntry
... six.print_(dm['ntry'], ntry)
... ntry += 1
... dm['ntry'] = ntry
... if ntry % 3:
......@@ -550,11 +566,11 @@ def test_invalidateCache():
and the cache will have been cleared:
>>> print connection.root()['a']._p_changed
>>> print(connection.root()['a']._p_changed)
None
>>> print connection.root()['b']._p_changed
>>> print(connection.root()['b']._p_changed)
None
>>> print connection.root()['c']._p_changed
>>> print(connection.root()['c']._p_changed)
None
But we'll be able to access data again:
......@@ -659,8 +675,8 @@ implementation of checkCurrentSerialInTransaction.
>>> from ZODB.POSException import ReadConflictError
>>> bad = set()
>>> def checkCurrentSerialInTransaction(oid, serial, trans):
... print 'checkCurrentSerialInTransaction', repr(oid)
... if not trans == transaction.get(): print 'oops'
... six.print_('checkCurrentSerialInTransaction', repr(oid))
... if not trans == transaction.get(): print('oops')
... if oid in bad:
... raise ReadConflictError(oid=oid)
......@@ -729,8 +745,8 @@ The conflict error will cause the affected object to be invalidated:
The storage may raise it later:
>>> def checkCurrentSerialInTransaction(oid, serial, trans):
... if not trans == transaction.get(): print 'oops'
... print 'checkCurrentSerialInTransaction', repr(oid)
... if not trans == transaction.get(): print('oops')
... six.print_('checkCurrentSerialInTransaction', repr(oid))
... store.badness = ReadConflictError(oid=oid)
>>> def tpc_vote(t):
......@@ -861,9 +877,9 @@ class C_invalidations_of_new_objects_work_after_savepoint(Persistent):
self.settings = 1
def _p_invalidate(self):
print 'INVALIDATE', self.settings
print('INVALIDATE', self.settings)
Persistent._p_invalidate(self)
print self.settings # POSKeyError here
print(self.settings) # POSKeyError here
def abort_of_savepoint_creating_new_objects_w_exotic_invalidate_doesnt_break():
r"""
......@@ -894,7 +910,7 @@ After the abort, the oid and jar are None:
class Clp9460655(Persistent):
def __init__(self, word, id):
super(Clp9460655, self).__init__()
self.id = id
self.id = id
self._word = word
def lp9460655():
......@@ -918,7 +934,7 @@ def lp9460655():
... data[word.id] = word
>>> sv.rollback()
>>> print commonWords[1].id # raises POSKeyError
>>> print(commonWords[1].id) # raises POSKeyError
1
"""
......@@ -956,7 +972,7 @@ def lp485456_setattr_in_setstate_doesnt_cause_multiple_stores():
>>> conn = ZODB.connection(None)
>>> oldstore = conn._storage.store
>>> def store(oid, *args):
... print 'storing', repr(oid)
... six.print_('storing', repr(oid))
... return oldstore(oid, *args)
>>> conn._storage.store = store
......@@ -988,7 +1004,7 @@ Let's try some combinations with savepoints:
>>> oldspstore = conn._storage.store
>>> def store(oid, *args):
... print 'savepoint storing', repr(oid)
... six.print_('savepoint storing', repr(oid))
... return oldspstore(oid, *args)
>>> conn._storage.store = store
......@@ -1096,7 +1112,7 @@ class EstimatedSizeTests(ZODB.tests.util.TestCase):
" <mappingstorage />\n"
"</zodb>"
)
self.assertEqual(db.getCacheSizeBytes(), 0x1L << 33)
self.assertEqual(db.getCacheSizeBytes(), 0x1 << 33)
def test_cache_garbage_collection(self):
......@@ -1190,7 +1206,7 @@ class StubStorage:
def new_oid(self):
oid = str(self._oid)
self._oid += 1
return oid
return str(oid).encode()
def sortKey(self):
return 'StubStorage sortKey'
......@@ -1273,7 +1289,7 @@ class StubDatabase:
def test_suite():
s = unittest.makeSuite(ConnectionDotAdd, 'check')
s.addTest(doctest.DocTestSuite())
s.addTest(doctest.DocTestSuite(checker=checker))
s.addTest(unittest.makeSuite(TestConnectionInterface))
s.addTest(unittest.makeSuite(EstimatedSizeTests))
return s
......@@ -13,9 +13,19 @@
##############################################################################
import doctest
import persistent.mapping
import re
import transaction
import unittest
import ZODB.tests.util
from zope.testing import renormalizing
checker = renormalizing.RENormalizing([
# Python 3 bytes add a "b".
(re.compile("b('.*?')"), r"\1"),
# Python 3 adds module name to exceptions.
(re.compile("ZODB.POSException.ConnectionStateError"),
r"ConnectionStateError"),
])
def testAddingThenModifyThenAbort():
"""\
......@@ -185,8 +195,10 @@ def tearDown(test):
def test_suite():
return unittest.TestSuite((
doctest.DocFileSuite('testConnectionSavepoint.txt', tearDown=tearDown),
doctest.DocTestSuite(tearDown=tearDown),
doctest.DocFileSuite(
'testConnectionSavepoint.txt',
tearDown=tearDown, checker=checker),
doctest.DocTestSuite(tearDown=tearDown, checker=checker),
))
if __name__ == '__main__':
......
......@@ -64,6 +64,7 @@ Now, we'll define a validation function to validate an account:
And a function to apply entries. If the function fails in some unexpected
way, it rolls back all of its changes and prints the error:
>>> import six
>>> def apply_entries(entries):
... savepoint = transaction.savepoint()
... try:
......@@ -72,14 +73,14 @@ way, it rolls back all of its changes and prints the error:
... try:
... root[name+'-balance'] += amount
... validate_account(name)
... except ValueError, error:
... except ValueError as error:
... entry_savepoint.rollback()
... print 'Error', str(error)
... six.print_('Error', str(error))
... else:
... print 'Updated', name
... except Exception, error:
... six.print_('Updated', name)
... except Exception as error:
... savepoint.rollback()
... print 'Unexpected exception', error
... six.print_('Unexpected exception', error)
Now let's try applying some entries:
......
......@@ -98,7 +98,7 @@ def test_invalidateCache():
>>> c3 is db.open(transaction_manager=tm3)
True
>>> print c3.root()['a']._p_changed
>>> print(c3.root()['a']._p_changed)
None
>>> db.close()
......
......@@ -29,12 +29,20 @@ if os.environ.get('USE_ZOPE_TESTING_DOCTEST'):
else:
import doctest
import random
import re
import transaction
import unittest
import ZODB.DemoStorage
import ZODB.tests.hexstorage
import ZODB.tests.util
import ZODB.utils
from zope.testing import renormalizing
checker = renormalizing.RENormalizing([
# Python 3 adds module name to exceptions.
(re.compile("ZODB.POSException.POSKeyError"), r"POSKeyError"),
])
class DemoStorageTests(
StorageTestBase.StorageTestBase,
......@@ -137,13 +145,14 @@ def setUp(test):
def testSomeDelegation():
r"""
>>> import six
>>> class S:
... def __init__(self, name):
... self.name = name
... def registerDB(self, db):
... print self.name, db
... six.print_(self.name, db)
... def close(self):
... print self.name, 'closed'
... six.print_(self.name, 'closed')
... sortKey = getSize = __len__ = history = getTid = None
... tpc_finish = tpc_vote = tpc_transaction = None
... _lock_acquire = _lock_release = lambda self: None
......@@ -154,7 +163,7 @@ def testSomeDelegation():
... def new_oid(self):
... return '\0' * 8
... def tpc_begin(self, t, tid, status):
... print 'begin', tid, status
... six.print_('begin', tid, status)
... def tpc_abort(self, t):
... pass
......@@ -242,7 +251,7 @@ def load_before_base_storage_current():
def test_suite():
suite = unittest.TestSuite((
doctest.DocTestSuite(
setUp=setUp, tearDown=ZODB.tests.util.tearDown,
setUp=setUp, tearDown=ZODB.tests.util.tearDown, checker=checker
),
doctest.DocFileSuite(
'../DemoStorage.test',
......
......@@ -11,7 +11,6 @@
# FOR A PARTICULAR PURPOSE.
#
##############################################################################
import cPickle
import doctest
import os
if os.environ.get('USE_ZOPE_TESTING_DOCTEST'):
......@@ -34,6 +33,14 @@ from ZODB.tests import RevisionStorage, PersistentStorage, MTStorage
from ZODB.tests import ReadOnlyStorage, RecoveryStorage
from ZODB.tests.StorageTestBase import MinPO, zodb_pickle
try:
import cPickle
except ImportError:
# Py3
import pickle as cPickle
class FileStorageTests(
StorageTestBase.StorageTestBase,
BasicStorage.BasicStorage,
......@@ -251,7 +258,7 @@ class FileStorageTests(
# NameError: global name 's' is not defined
try:
self._storage.pack(time.time(), referencesf)
except CorruptedError, detail:
except CorruptedError as detail:
self.assert_("redundant transaction length does not match "
"initial transaction length" in str(detail))
else:
......
......@@ -26,9 +26,21 @@ import transaction
from transaction import Transaction
import ZODB
from ZODB.MappingStorage import MappingStorage
import cPickle
import cStringIO
import sys
import six
try:
import cPickle
except ImportError:
# Py3
import pickle as cPickle
try:
import cStringIO
except ImportError:
# Py3
import io as cStringIO
# This pickle contains a persistent mapping pickle created from the
# old code.
......@@ -126,15 +138,15 @@ class PMTests(unittest.TestCase):
self.assertEqual(items,
[('a', 2), ('b', 3), ('name', 'bob'), ('x', 1)])
keys = list(m.iterkeys())
keys = list(six.iterkeys(m))
keys.sort()
self.assertEqual(keys, ['a', 'b', 'name', 'x'])
values = list(m.itervalues())
values = list(six.itervalues(m))
values.sort()
self.assertEqual(values, [1, 2, 3, 'bob'])
items = list(m.iteritems())
items = list(six.iteritems(m))
items.sort()
self.assertEqual(items,
[('a', 2), ('b', 3), ('name', 'bob'), ('x', 1)])
......@@ -157,7 +169,7 @@ class PMTests(unittest.TestCase):
keylist = []
while 1:
try:
key = i.next()
key = six.advance_iterator(i)
except StopIteration:
break
keylist.append(key)
......
......@@ -18,7 +18,6 @@ import os
import random
import sys
import unittest
import StringIO
import ZODB
import ZODB.tests.util
......@@ -28,6 +27,12 @@ import ZODB.fsrecover
from persistent.mapping import PersistentMapping
import transaction
try:
import StringIO
except ImportError:
# Py3
import io as StringIO
class RecoverTest(ZODB.tests.util.TestCase):
path = None
......
......@@ -12,13 +12,23 @@
#
##############################################################################
import doctest
import cPickle
import cStringIO as StringIO
import sys
import unittest
from ZODB import serialize
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
try:
from cStringIO import StringIO as BytesIO
except ImportError:
# Py3
from io import BytesIO
class ClassWithNewargs(int):
def __new__(cls, value):
......@@ -32,8 +42,8 @@ class ClassWithoutNewargs(object):
self.value = value
def make_pickle(ob):
sio = StringIO.StringIO()
p = cPickle.Pickler(sio, 1)
sio = BytesIO()
p = pickle.Pickler(sio, 1)
p.dump(ob)
return sio.getvalue()
......
......@@ -12,21 +12,35 @@
#
##############################################################################
"""Test the routines to convert between long and 64-bit strings"""
from persistent import Persistent
import doctest
import random
import re
import unittest
from persistent import Persistent
from zope.testing import renormalizing
from ZODB.utils import U64, p64, u64
try:
long(1)
except NameError:
# Py3
long = int
NUM = 100
from ZODB.utils import U64, p64, u64
checker = renormalizing.RENormalizing([
# Python 3 bytes add a "b".
(re.compile("b('.*?')"),
r"\1"),
])
class TestUtils(unittest.TestCase):
small = [random.randrange(1, 1L<<32, int=long)
small = [random.randrange(1, 1<<32, int=long)
for i in range(NUM)]
large = [random.randrange(1L<<32, 1L<<64, int=long)
large = [random.randrange(1<<32, 1<<64, int=long)
for i in range(NUM)]
all = small + large
......@@ -40,11 +54,11 @@ class TestUtils(unittest.TestCase):
def checkKnownConstants(self):
self.assertEquals("\000\000\000\000\000\000\000\001", p64(1))
self.assertEquals("\000\000\000\001\000\000\000\000", p64(1L<<32))
self.assertEquals("\000\000\000\001\000\000\000\000", p64(1<<32))
self.assertEquals(u64("\000\000\000\000\000\000\000\001"), 1)
self.assertEquals(U64("\000\000\000\000\000\000\000\001"), 1)
self.assertEquals(u64("\000\000\000\001\000\000\000\000"), 1L<<32)
self.assertEquals(U64("\000\000\000\001\000\000\000\000"), 1L<<32)
self.assertEquals(u64("\000\000\000\001\000\000\000\000"), 1<<32)
self.assertEquals(U64("\000\000\000\001\000\000\000\000"), 1<<32)
def checkPersistentIdHandlesDescriptor(self):
from ZODB.serialize import ObjectWriter
......@@ -63,7 +77,11 @@ class TestUtils(unittest.TestCase):
from ZODB.serialize import ObjectWriter
from ZODB.POSException import ConflictError
from ZODB.tests.MinPO import MinPO
import cPickle as pickle
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
obj = MinPO()
data = ObjectWriter().serialize(obj)
......@@ -81,7 +99,7 @@ class TestUtils(unittest.TestCase):
# Verify that building ConflictError doesn't get ImportError.
try:
raise ConflictError(object=obj, data=data)
except ConflictError, detail:
except ConflictError as detail:
# And verify that the msg names the impossible path.
self.assert_('path.that.does.not.exist.likewise.the.class' in
str(detail))
......@@ -92,5 +110,5 @@ class TestUtils(unittest.TestCase):
def test_suite():
suite = unittest.TestSuite()
suite.addTest(unittest.makeSuite(TestUtils, 'check'))
suite.addTest(doctest.DocFileSuite('../utils.txt'))
suite.addTest(doctest.DocFileSuite('../utils.txt', checker=checker))
return suite
......@@ -24,6 +24,8 @@ import ZODB
import ZODB.FileStorage
import ZODB.MappingStorage
import ZODB.tests.util
from six.moves import map
from six.moves import zip
class P(Persistent):
pass
......@@ -106,8 +108,8 @@ class ZODBTests(ZODB.tests.util.TestCase):
l1.sort()
l2 = list(ob2.items())
l2.sort()
l1 = map(lambda (k, v): (k, v[0]), l1)
l2 = map(lambda (k, v): (k, v[0]), l2)
l1 = map(lambda k_v: (k_v[0], k_v[1][0]), l1)
l2 = map(lambda k_v1: (k_v1[0], k_v1[1][0]), l2)
self.assertEqual(l1, l2)
self.assert_(ob._p_oid != ob2._p_oid)
self.assertEqual(ob._p_jar, ob2._p_jar)
......@@ -115,7 +117,7 @@ class ZODBTests(ZODB.tests.util.TestCase):
for v in ob.values():
oids[v._p_oid] = 1
for v in ob2.values():
assert not oids.has_key(v._p_oid), (
assert v._p_oid not in oids, (
'Did not fully separate duplicate from original')
transaction.commit()
......@@ -157,7 +159,7 @@ class ZODBTests(ZODB.tests.util.TestCase):
try:
r1 = conn1.root()
r2 = conn2.root()
if r1.has_key('item'):
if 'item' in r1:
del r1['item']
tm1.get().commit()
r1.get('item')
......
......@@ -12,9 +12,7 @@
#
##############################################################################
from pickle import Pickler
from pickle import Unpickler
from StringIO import StringIO
from pickle import Pickler, Unpickler
from ZODB.blob import Blob
from ZODB.DB import DB
from ZODB.FileStorage import FileStorage
......@@ -42,6 +40,14 @@ import ZODB.tests.StorageTestBase
import ZODB.tests.util
import zope.testing.renormalizing
try:
from StringIO import StringIO
except ImportError:
# Py3
from io import StringIO
def new_time():
"""Create a _new_ time stamp.
......
......@@ -18,6 +18,7 @@ import unittest
from ZODB.fsIndex import fsIndex
from ZODB.utils import p64, z64
from ZODB.tests.util import setUp, tearDown
import six
class Test(unittest.TestCase):
......@@ -26,7 +27,7 @@ class Test(unittest.TestCase):
self.index = fsIndex()
for i in range(200):
self.index[p64(i * 1000)] = (i * 1000L + 1)
self.index[p64(i * 1000)] = (i * 1000 + 1)
def test__del__(self):
index = self.index
......@@ -50,7 +51,7 @@ class Test(unittest.TestCase):
index = self.index
for i in range(0,200):
self.assertEqual((i,index[p64(i*1000)]), (i,(i*1000L+1)))
self.assertEqual((i,index[p64(i*1000)]), (i,(i*1000+1)))
self.assertEqual(len(index), 200)
......@@ -69,17 +70,17 @@ class Test(unittest.TestCase):
d={}
for i in range(200):
d[p64(i*1000)]=(i*1000L+1)
d[p64(i*1000)]=(i*1000+1)
index.update(d)
for i in range(400,600):
d[p64(i*1000)]=(i*1000L+1)
d[p64(i*1000)]=(i*1000+1)
index.update(d)
for i in range(100, 500):
d[p64(i*1000)]=(i*1000L+2)
d[p64(i*1000)]=(i*1000+2)
index.update(d)
......@@ -95,7 +96,7 @@ class Test(unittest.TestCase):
for i, k in enumerate(keys):
self.assertEqual(k, p64(i * 1000))
keys = list(self.index.iterkeys())
keys = list(six.iterkeys(self.index))
keys.sort()
for i, k in enumerate(keys):
......@@ -108,30 +109,30 @@ class Test(unittest.TestCase):
self.assertEqual(k, p64(i * 1000))
def testValues(self):
values = list(self.index.itervalues())
values = list(six.itervalues(self.index))
values.sort()
for i, v in enumerate(values):
self.assertEqual(v, (i * 1000L + 1))
self.assertEqual(v, (i * 1000 + 1))
values = self.index.values()
values.sort()
for i, v in enumerate(values):
self.assertEqual(v, (i * 1000L + 1))
self.assertEqual(v, (i * 1000 + 1))
def testItems(self):
items = list(self.index.iteritems())
items = list(six.iteritems(self.index))
items.sort()
for i, item in enumerate(items):
self.assertEqual(item, (p64(i * 1000), (i * 1000L + 1)))
self.assertEqual(item, (p64(i * 1000), (i * 1000 + 1)))
items = self.index.items()
items.sort()
for i, item in enumerate(items):
self.assertEqual(item, (p64(i * 1000), (i * 1000L + 1)))
self.assertEqual(item, (p64(i * 1000), (i * 1000 + 1)))
def testMaxKey(self):
index = self.index
......@@ -214,7 +215,11 @@ Note that we pass a file position, which gets saved with the index data.
If we save the data in the old format, we can still read it:
>>> import cPickle
>>> try:
... import cPickle
... except ImportError:
... # Py3
... import pickle as cPickle
>>> cPickle.dump(dict(pos=42, index=index), open('old', 'wb'), 1)
>>> info = fsIndex.load('old')
>>> info['pos']
......
......@@ -74,7 +74,7 @@ an invalidation arrives.
>>> cn = db.open()
>>> print cn._txn_time
>>> print(cn._txn_time)
None
>>> cn.invalidate(100, dict.fromkeys([1, 2]))
>>> cn._txn_time
......@@ -94,7 +94,7 @@ but that doesn't work unless an object is modified. sync() will abort
a transaction and process invalidations.
>>> cn.sync()
>>> print cn._txn_time # the high-water mark got reset to None
>>> print(cn._txn_time) # the high-water mark got reset to None
None
Basic functionality
......@@ -152,7 +152,7 @@ ConflictError: database conflict error (oid 0x01, class ZODB.tests.MinPO.MinPO)
This example will demonstrate that we can commit a transaction if we only
modify current revisions.
>>> print cn2._txn_time
>>> print(cn2._txn_time)
None
>>> r1 = cn1.root()
......@@ -166,7 +166,7 @@ True
>>> r2["b"].value
3
>>> tm2.get().commit()
>>> print cn2._txn_time
>>> print(cn2._txn_time)
None
Object cache
......@@ -401,8 +401,18 @@ True
>>> ts.count
1
"""
import doctest
import re
from zope.testing import renormalizing
checker = renormalizing.RENormalizing([
# Python 3 bytes add a "b".
(re.compile("b('.*?')"), r"\1"),
# Python 3 adds module name to exceptions.
(re.compile("ZODB.POSException.ConflictError"), r"ConflictError"),
(re.compile("ZODB.POSException.ReadConflictError"), r"ReadConflictError"),
])
def test_suite():
return doctest.DocTestSuite()
return doctest.DocTestSuite(checker=checker)
......@@ -121,7 +121,7 @@ def assert_deprecated(func, warning_text=''):
def wait(func=None, timeout=30):
if func is None:
return lambda f: wait(f, timeout)
for i in xrange(int(timeout*100)):
for i in range(int(timeout*100)):
if func():
return
time.sleep(.01)
......
......@@ -42,14 +42,14 @@ def transact(f, note=None, retries=5):
n -= 1
try:
r = f(*args, **kwargs)
except ReadConflictError, msg:
except ReadConflictError as msg:
transaction.abort()
if not n:
raise
continue
try:
_commit(note)
except ConflictError, msg:
except ConflictError as msg:
transaction.abort()
if not n:
raise
......
......@@ -11,18 +11,28 @@
# FOR A PARTICULAR PURPOSE
#
##############################################################################
import sys
import time
import struct
from struct import pack, unpack
from binascii import hexlify, unhexlify
import cPickle as pickle
from cStringIO import StringIO
import warnings
from tempfile import mkstemp
import os
try:
import cPickle as pickle
except ImportError:
# Py3
import pickle
try:
from cStringIO import StringIO
except ImportError:
# Py3
from io import StringIO
from persistent.TimeStamp import TimeStamp
__all__ = ['z64',
......@@ -66,6 +76,42 @@ def deprecated38(msg):
warnings.warn("This will be removed in ZODB 3.8:\n%s" % msg,
DeprecationWarning, stacklevel=3)
if sys.version_info[0] < 3:
bytes = str
def as_bytes(obj):
"Convert obj into bytes"
return str(obj)
def as_text(bytes):
"Convert bytes into string"
return bytes
# Convert an element of a bytes object into an int
byte_ord = ord
byte_chr = chr
else:
import builtins
bytes = builtins.bytes
def as_bytes(obj):
if isinstance(obj, bytes):
# invoking str on a bytes object gives its repr()
return obj
return str(obj).encode("ascii")
def as_text(bytes):
return bytes.decode("ascii")
def byte_ord(byte):
return byte # elements of bytes are already ints
def byte_chr(int):
return bytes((int,))
z64 = b'\0' * 8
assert sys.hexversion >= 0x02030000
......@@ -121,15 +167,15 @@ def newTid(old):
def oid_repr(oid):
if isinstance(oid, str) and len(oid) == 8:
if isinstance(oid, bytes) and len(oid) == 8:
# Convert to hex and strip leading zeroes.
as_hex = hexlify(oid).lstrip('0')
as_hex = hexlify(oid).lstrip(b'0')
# Ensure two characters per input byte.
if len(as_hex) & 1:
as_hex = '0' + as_hex
elif as_hex == '':
as_hex = '00'
return '0x' + as_hex
as_hex = b'0' + as_hex
elif as_hex == b'':
as_hex = b'00'
return '0x' + as_hex.decode()
else:
return repr(oid)
......@@ -204,7 +250,7 @@ def get_pickle_metadata(data):
u = pickle.Unpickler(f)
try:
class_info = u.load()
except Exception, err:
except Exception as err:
return '', ''
if isinstance(class_info, tuple):
if isinstance(class_info[0], tuple):
......@@ -228,19 +274,20 @@ def mktemp(dir=None):
class Locked(object):
def __init__(self, func, inst=None, class_=None, preconditions=()):
self.im_func = func
self.im_self = inst
self.im_class = class_
self.__func__ = func
self.__self__ = inst
self.__self_class__ = class_
self.preconditions = preconditions
def __get__(self, inst, class_):
return self.__class__(self.im_func, inst, class_, self.preconditions)
return self.__class__(
self.__func__, inst, class_, self.preconditions)
def __call__(self, *args, **kw):
inst = self.im_self
inst = self.__self__
if inst is None:
inst = args[0]
func = self.im_func.__get__(self.im_self, self.im_class)
func = self.__func__.__get__(self.__self__, self.__self_class__)
inst._lock_acquire()
try:
......
This diff is collapsed.
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment