Get rid of tpool_reraise
As best I can tell, eventlet already does (and always has done) the right thing, and we were just bad at catching Timeouts. For some history: https://github.com/openstack/swift/commit/5db3cb3 https://github.com/openstack/swift/commit/2b3aab8 https://github.com/openstack/swift/commit/da0e013 Change-Id: Iad8109c4a03f006a89e55373cf3ca867d724b3e1 Related-Bug: 1647804
This commit is contained in:
parent
eb9a21b22b
commit
dc8d1c964a
@ -59,7 +59,7 @@ import eventlet.debug
|
|||||||
import eventlet.greenthread
|
import eventlet.greenthread
|
||||||
import eventlet.patcher
|
import eventlet.patcher
|
||||||
import eventlet.semaphore
|
import eventlet.semaphore
|
||||||
from eventlet import GreenPool, sleep, Timeout, tpool
|
from eventlet import GreenPool, sleep, Timeout
|
||||||
from eventlet.green import socket, threading
|
from eventlet.green import socket, threading
|
||||||
from eventlet.hubs import trampoline
|
from eventlet.hubs import trampoline
|
||||||
import eventlet.queue
|
import eventlet.queue
|
||||||
@ -3771,21 +3771,6 @@ class Spliterator(object):
|
|||||||
self._iterator_in_progress = False
|
self._iterator_in_progress = False
|
||||||
|
|
||||||
|
|
||||||
def tpool_reraise(func, *args, **kwargs):
|
|
||||||
"""
|
|
||||||
Hack to work around Eventlet's tpool not catching and reraising Timeouts.
|
|
||||||
"""
|
|
||||||
def inner():
|
|
||||||
try:
|
|
||||||
return func(*args, **kwargs)
|
|
||||||
except BaseException as err:
|
|
||||||
return err
|
|
||||||
resp = tpool.execute(inner)
|
|
||||||
if isinstance(resp, BaseException):
|
|
||||||
raise resp
|
|
||||||
return resp
|
|
||||||
|
|
||||||
|
|
||||||
def ismount(path):
|
def ismount(path):
|
||||||
"""
|
"""
|
||||||
Test whether a path is a mount point. This will catch any
|
Test whether a path is a mount point. This will catch any
|
||||||
|
@ -50,7 +50,7 @@ from contextlib import contextmanager
|
|||||||
from collections import defaultdict
|
from collections import defaultdict
|
||||||
from datetime import timedelta
|
from datetime import timedelta
|
||||||
|
|
||||||
from eventlet import Timeout
|
from eventlet import Timeout, tpool
|
||||||
from eventlet.hubs import trampoline
|
from eventlet.hubs import trampoline
|
||||||
import six
|
import six
|
||||||
from pyeclib.ec_iface import ECDriverError, ECInvalidFragmentMetadata, \
|
from pyeclib.ec_iface import ECDriverError, ECInvalidFragmentMetadata, \
|
||||||
@ -64,7 +64,7 @@ from swift.common.utils import mkdirs, Timestamp, \
|
|||||||
fsync_dir, drop_buffer_cache, lock_path, write_pickle, \
|
fsync_dir, drop_buffer_cache, lock_path, write_pickle, \
|
||||||
config_true_value, listdir, split_path, remove_file, \
|
config_true_value, listdir, split_path, remove_file, \
|
||||||
get_md5_socket, F_SETPIPE_SZ, decode_timestamps, encode_timestamps, \
|
get_md5_socket, F_SETPIPE_SZ, decode_timestamps, encode_timestamps, \
|
||||||
tpool_reraise, MD5_OF_EMPTY_STRING, link_fd_to_path, o_tmpfile_supported, \
|
MD5_OF_EMPTY_STRING, link_fd_to_path, o_tmpfile_supported, \
|
||||||
O_TMPFILE, makedirs_count, replace_partition_in_path
|
O_TMPFILE, makedirs_count, replace_partition_in_path
|
||||||
from swift.common.splice import splice, tee
|
from swift.common.splice import splice, tee
|
||||||
from swift.common.exceptions import DiskFileQuarantined, DiskFileNotExist, \
|
from swift.common.exceptions import DiskFileQuarantined, DiskFileNotExist, \
|
||||||
@ -1419,7 +1419,7 @@ class BaseDiskFileManager(object):
|
|||||||
partition_path = get_part_path(dev_path, policy, partition)
|
partition_path = get_part_path(dev_path, policy, partition)
|
||||||
if not os.path.exists(partition_path):
|
if not os.path.exists(partition_path):
|
||||||
mkdirs(partition_path)
|
mkdirs(partition_path)
|
||||||
_junk, hashes = tpool_reraise(
|
_junk, hashes = tpool.execute(
|
||||||
self._get_hashes, device, partition, policy, recalculate=suffixes)
|
self._get_hashes, device, partition, policy, recalculate=suffixes)
|
||||||
return hashes
|
return hashes
|
||||||
|
|
||||||
@ -1598,7 +1598,7 @@ class BaseDiskFileWriter(object):
|
|||||||
# For large files sync every 512MB (by default) written
|
# For large files sync every 512MB (by default) written
|
||||||
diff = self._upload_size - self._last_sync
|
diff = self._upload_size - self._last_sync
|
||||||
if diff >= self._bytes_per_sync:
|
if diff >= self._bytes_per_sync:
|
||||||
tpool_reraise(fdatasync, self._fd)
|
tpool.execute(fdatasync, self._fd)
|
||||||
drop_buffer_cache(self._fd, self._last_sync, diff)
|
drop_buffer_cache(self._fd, self._last_sync, diff)
|
||||||
self._last_sync = self._upload_size
|
self._last_sync = self._upload_size
|
||||||
|
|
||||||
@ -1678,7 +1678,7 @@ class BaseDiskFileWriter(object):
|
|||||||
metadata['name'] = self._name
|
metadata['name'] = self._name
|
||||||
target_path = join(self._datadir, filename)
|
target_path = join(self._datadir, filename)
|
||||||
|
|
||||||
tpool_reraise(self._finalize_put, metadata, target_path, cleanup)
|
tpool.execute(self._finalize_put, metadata, target_path, cleanup)
|
||||||
|
|
||||||
def put(self, metadata):
|
def put(self, metadata):
|
||||||
"""
|
"""
|
||||||
@ -2967,7 +2967,7 @@ class ECDiskFileWriter(BaseDiskFileWriter):
|
|||||||
durable_data_file_path = os.path.join(
|
durable_data_file_path = os.path.join(
|
||||||
self._datadir, self.manager.make_on_disk_filename(
|
self._datadir, self.manager.make_on_disk_filename(
|
||||||
timestamp, '.data', self._diskfile._frag_index, durable=True))
|
timestamp, '.data', self._diskfile._frag_index, durable=True))
|
||||||
tpool_reraise(
|
tpool.execute(
|
||||||
self._finalize_durable, data_file_path, durable_data_file_path)
|
self._finalize_durable, data_file_path, durable_data_file_path)
|
||||||
|
|
||||||
def put(self, metadata):
|
def put(self, metadata):
|
||||||
|
@ -32,7 +32,7 @@ from swift import gettext_ as _
|
|||||||
from swift.common.utils import (
|
from swift.common.utils import (
|
||||||
whataremyips, unlink_older_than, compute_eta, get_logger,
|
whataremyips, unlink_older_than, compute_eta, get_logger,
|
||||||
dump_recon_cache, mkdirs, config_true_value,
|
dump_recon_cache, mkdirs, config_true_value,
|
||||||
tpool_reraise, GreenAsyncPile, Timestamp, remove_file,
|
GreenAsyncPile, Timestamp, remove_file,
|
||||||
load_recon_cache, parse_override_options, distribute_evenly,
|
load_recon_cache, parse_override_options, distribute_evenly,
|
||||||
PrefixLoggerAdapter)
|
PrefixLoggerAdapter)
|
||||||
from swift.common.header_key_dict import HeaderKeyDict
|
from swift.common.header_key_dict import HeaderKeyDict
|
||||||
@ -616,7 +616,7 @@ class ObjectReconstructor(Daemon):
|
|||||||
def _get_hashes(self, device, partition, policy, recalculate=None,
|
def _get_hashes(self, device, partition, policy, recalculate=None,
|
||||||
do_listdir=False):
|
do_listdir=False):
|
||||||
df_mgr = self._df_router[policy]
|
df_mgr = self._df_router[policy]
|
||||||
hashed, suffix_hashes = tpool_reraise(
|
hashed, suffix_hashes = tpool.execute(
|
||||||
df_mgr._get_hashes, device, partition, policy,
|
df_mgr._get_hashes, device, partition, policy,
|
||||||
recalculate=recalculate, do_listdir=do_listdir)
|
recalculate=recalculate, do_listdir=do_listdir)
|
||||||
self.logger.update_stats('suffix.hashes', hashed)
|
self.logger.update_stats('suffix.hashes', hashed)
|
||||||
|
@ -34,7 +34,7 @@ from swift.common.ring.utils import is_local_device
|
|||||||
from swift.common.utils import whataremyips, unlink_older_than, \
|
from swift.common.utils import whataremyips, unlink_older_than, \
|
||||||
compute_eta, get_logger, dump_recon_cache, \
|
compute_eta, get_logger, dump_recon_cache, \
|
||||||
rsync_module_interpolation, mkdirs, config_true_value, \
|
rsync_module_interpolation, mkdirs, config_true_value, \
|
||||||
tpool_reraise, config_auto_int_value, storage_directory, \
|
config_auto_int_value, storage_directory, \
|
||||||
load_recon_cache, PrefixLoggerAdapter, parse_override_options, \
|
load_recon_cache, PrefixLoggerAdapter, parse_override_options, \
|
||||||
distribute_evenly
|
distribute_evenly
|
||||||
from swift.common.bufferedhttp import http_connect
|
from swift.common.bufferedhttp import http_connect
|
||||||
@ -610,7 +610,7 @@ class ObjectReplicator(Daemon):
|
|||||||
begin = time.time()
|
begin = time.time()
|
||||||
df_mgr = self._df_router[job['policy']]
|
df_mgr = self._df_router[job['policy']]
|
||||||
try:
|
try:
|
||||||
hashed, local_hash = tpool_reraise(
|
hashed, local_hash = tpool.execute(
|
||||||
df_mgr._get_hashes, job['device'],
|
df_mgr._get_hashes, job['device'],
|
||||||
job['partition'], job['policy'],
|
job['partition'], job['policy'],
|
||||||
do_listdir=_do_listdir(
|
do_listdir=_do_listdir(
|
||||||
@ -664,7 +664,7 @@ class ObjectReplicator(Daemon):
|
|||||||
if not suffixes:
|
if not suffixes:
|
||||||
stats.hashmatch += 1
|
stats.hashmatch += 1
|
||||||
continue
|
continue
|
||||||
hashed, recalc_hash = tpool_reraise(
|
hashed, recalc_hash = tpool.execute(
|
||||||
df_mgr._get_hashes,
|
df_mgr._get_hashes,
|
||||||
job['device'], job['partition'], job['policy'],
|
job['device'], job['partition'], job['policy'],
|
||||||
recalculate=suffixes)
|
recalculate=suffixes)
|
||||||
|
@ -3237,18 +3237,6 @@ cluster_dfw1 = http://dfw1.host/v1/
|
|||||||
self.assertEqual(fallocate_value, 10.000)
|
self.assertEqual(fallocate_value, 10.000)
|
||||||
self.assertTrue(is_percent)
|
self.assertTrue(is_percent)
|
||||||
|
|
||||||
def test_tpool_reraise(self):
|
|
||||||
with patch.object(utils.tpool, 'execute', lambda f: f()):
|
|
||||||
self.assertTrue(
|
|
||||||
utils.tpool_reraise(MagicMock(return_value='test1')), 'test1')
|
|
||||||
self.assertRaises(
|
|
||||||
Exception,
|
|
||||||
utils.tpool_reraise, MagicMock(side_effect=Exception('test2')))
|
|
||||||
self.assertRaises(
|
|
||||||
BaseException,
|
|
||||||
utils.tpool_reraise,
|
|
||||||
MagicMock(side_effect=BaseException('test3')))
|
|
||||||
|
|
||||||
def test_lock_file(self):
|
def test_lock_file(self):
|
||||||
flags = os.O_CREAT | os.O_RDWR
|
flags = os.O_CREAT | os.O_RDWR
|
||||||
with NamedTemporaryFile(delete=False) as nt:
|
with NamedTemporaryFile(delete=False) as nt:
|
||||||
|
@ -1798,10 +1798,10 @@ class TestObjectReplicator(unittest.TestCase):
|
|||||||
self.replicator.sync_method.assert_called_once_with(
|
self.replicator.sync_method.assert_called_once_with(
|
||||||
'node', 'job', 'suffixes')
|
'node', 'job', 'suffixes')
|
||||||
|
|
||||||
@mock.patch('swift.obj.replicator.tpool_reraise')
|
@mock.patch('swift.obj.replicator.tpool.execute')
|
||||||
@mock.patch('swift.obj.replicator.http_connect', autospec=True)
|
@mock.patch('swift.obj.replicator.http_connect', autospec=True)
|
||||||
@mock.patch('swift.obj.replicator._do_listdir')
|
@mock.patch('swift.obj.replicator._do_listdir')
|
||||||
def test_update(self, mock_do_listdir, mock_http, mock_tpool_reraise):
|
def test_update(self, mock_do_listdir, mock_http, mock_tpool_execute):
|
||||||
|
|
||||||
def set_default(self):
|
def set_default(self):
|
||||||
self.replicator.suffix_count = 0
|
self.replicator.suffix_count = 0
|
||||||
@ -1813,7 +1813,7 @@ class TestObjectReplicator(unittest.TestCase):
|
|||||||
|
|
||||||
self.headers = {'Content-Length': '0',
|
self.headers = {'Content-Length': '0',
|
||||||
'user-agent': 'object-replicator %s' % os.getpid()}
|
'user-agent': 'object-replicator %s' % os.getpid()}
|
||||||
mock_tpool_reraise.return_value = (0, {})
|
mock_tpool_execute.return_value = (0, {})
|
||||||
|
|
||||||
all_jobs = self.replicator.collect_jobs()
|
all_jobs = self.replicator.collect_jobs()
|
||||||
jobs = [job for job in all_jobs if not job['delete']]
|
jobs = [job for job in all_jobs if not job['delete']]
|
||||||
@ -1868,7 +1868,7 @@ class TestObjectReplicator(unittest.TestCase):
|
|||||||
mock_http.reset_mock()
|
mock_http.reset_mock()
|
||||||
self.logger.clear()
|
self.logger.clear()
|
||||||
mock_do_listdir.assert_has_calls(expected_listdir_calls)
|
mock_do_listdir.assert_has_calls(expected_listdir_calls)
|
||||||
mock_tpool_reraise.assert_has_calls(expected_tpool_calls)
|
mock_tpool_execute.assert_has_calls(expected_tpool_calls)
|
||||||
mock_do_listdir.side_effect = None
|
mock_do_listdir.side_effect = None
|
||||||
mock_do_listdir.return_value = False
|
mock_do_listdir.return_value = False
|
||||||
# Check incorrect http_connect with status 400 != HTTP_OK
|
# Check incorrect http_connect with status 400 != HTTP_OK
|
||||||
@ -1920,7 +1920,7 @@ class TestObjectReplicator(unittest.TestCase):
|
|||||||
self.logger.clear()
|
self.logger.clear()
|
||||||
|
|
||||||
# Check successful http_connect and sync for local node
|
# Check successful http_connect and sync for local node
|
||||||
mock_tpool_reraise.return_value = (1, {'a83': 'ba47fd314242ec8c'
|
mock_tpool_execute.return_value = (1, {'a83': 'ba47fd314242ec8c'
|
||||||
'7efb91f5d57336e4'})
|
'7efb91f5d57336e4'})
|
||||||
resp.read.return_value = pickle.dumps({'a83': 'c130a2c17ed45102a'
|
resp.read.return_value = pickle.dumps({'a83': 'c130a2c17ed45102a'
|
||||||
'ada0f4eee69494ff'})
|
'ada0f4eee69494ff'})
|
||||||
|
Loading…
x
Reference in New Issue
Block a user