Params refactor, simplified (#2300)

* always c++

* Create C++ params class

* get works

* tests hang now

* passes tests

* cleanup string formatting

* handle interrupt in blocking read

* fix memory leak

* remove unused constructor

* Use delete_db_value directly

* Rename put -> write_db_value

* filename cleanup

* no semicolons in cython

* Update common/SConscript

Co-authored-by: Adeeb Shihadeh <adeebshihadeh@gmail.com>

* add std::string version of delete_db_value

* This is handled

* cleanup encoding

* Add clear method to clear all

* add persistent params

* fix android build

* Should be called clear_all

* only import params when needed

* set params path on manager import

* recusrively create directories

* Fix function order

* cleanup mkdirp

Co-authored-by: Adeeb Shihadeh <adeebshihadeh@gmail.com>
Co-authored-by: Comma Device <device@comma.ai>
albatross
Willem Melching 2020-10-13 16:23:23 +02:00 committed by GitHub
parent a4ccfcffe8
commit 2e182e5c57
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
20 changed files with 417 additions and 577 deletions

View File

@ -4,3 +4,11 @@ Import('env', 'cython_dependencies')
env.Command(['common_pyx.so', 'clock.cpp'],
cython_dependencies + ['common_pyx_setup.py', 'clock.pyx'],
"cd common && python3 common_pyx_setup.py build_ext --inplace")
# Build cython params module
env.Command(['params_pyx.so', 'params_pyx.cpp'],
cython_dependencies + [
'params_pyx_setup.py', 'params_pyx.pyx', 'params_pxd.pxd',
'#selfdrive/common/params.cc', '#selfdrive/common/params.h',
'#selfdrive/common/util.c', '#selfdrive/common/util.h'],
"cd common && python3 params_pyx_setup.py build_ext --inplace")

413
common/params.py 100755 → 100644
View File

@ -1,409 +1,4 @@
#!/usr/bin/env python3
"""ROS has a parameter server, we have files.
The parameter store is a persistent key value store, implemented as a directory with a writer lock.
On Android, we store params under params_dir = /data/params. The writer lock is a file
"<params_dir>/.lock" taken using flock(), and data is stored in a directory symlinked to by
"<params_dir>/d".
Each key, value pair is stored as a file with named <key> with contents <value>, located in
<params_dir>/d/<key>
Readers of a single key can just open("<params_dir>/d/<key>") and read the file contents.
Readers who want a consistent snapshot of multiple keys should take the lock.
Writers should take the lock before modifying anything. Writers should also leave the DB in a
consistent state after a crash. The implementation below does this by copying all params to a temp
directory <params_dir>/<tmp>, then atomically symlinking <params_dir>/<d> to <params_dir>/<tmp>
before deleting the old <params_dir>/<d> directory.
Writers that only modify a single key can simply take the lock, then swap the corresponding value
file in place without messing with <params_dir>/d.
"""
import time
import os
import errno
import shutil
import fcntl
import tempfile
import threading
from enum import Enum
from common.basedir import PARAMS
def mkdirs_exists_ok(path):
try:
os.makedirs(path)
except OSError:
if not os.path.isdir(path):
raise
class TxType(Enum):
PERSISTENT = 1
CLEAR_ON_MANAGER_START = 2
CLEAR_ON_PANDA_DISCONNECT = 3
class UnknownKeyName(Exception):
pass
keys = {
"AccessToken": [TxType.CLEAR_ON_MANAGER_START],
"AthenadPid": [TxType.PERSISTENT],
"CalibrationParams": [TxType.PERSISTENT],
"CarBatteryCapacity": [TxType.PERSISTENT],
"CarParams": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"CarParamsCache": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"CarVin": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"CommunityFeaturesToggle": [TxType.PERSISTENT],
"CompletedTrainingVersion": [TxType.PERSISTENT],
"DisablePowerDown": [TxType.PERSISTENT],
"DisableUpdates": [TxType.PERSISTENT],
"DoUninstall": [TxType.CLEAR_ON_MANAGER_START],
"DongleId": [TxType.PERSISTENT],
"GitBranch": [TxType.PERSISTENT],
"GitCommit": [TxType.PERSISTENT],
"GitRemote": [TxType.PERSISTENT],
"GithubSshKeys": [TxType.PERSISTENT],
"HasAcceptedTerms": [TxType.PERSISTENT],
"HasCompletedSetup": [TxType.PERSISTENT],
"IsDriverViewEnabled": [TxType.CLEAR_ON_MANAGER_START],
"IsLdwEnabled": [TxType.PERSISTENT],
"IsMetric": [TxType.PERSISTENT],
"IsOffroad": [TxType.CLEAR_ON_MANAGER_START],
"IsRHD": [TxType.PERSISTENT],
"IsTakingSnapshot": [TxType.CLEAR_ON_MANAGER_START],
"IsUpdateAvailable": [TxType.CLEAR_ON_MANAGER_START],
"IsUploadRawEnabled": [TxType.PERSISTENT],
"LastAthenaPingTime": [TxType.PERSISTENT],
"LastUpdateTime": [TxType.PERSISTENT],
"LastUpdateException": [TxType.PERSISTENT],
"LiveParameters": [TxType.PERSISTENT],
"OpenpilotEnabledToggle": [TxType.PERSISTENT],
"LaneChangeEnabled": [TxType.PERSISTENT],
"PandaFirmware": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"PandaFirmwareHex": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"PandaDongleId": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"Passive": [TxType.PERSISTENT],
"RecordFront": [TxType.PERSISTENT],
"ReleaseNotes": [TxType.PERSISTENT],
"ShouldDoUpdate": [TxType.CLEAR_ON_MANAGER_START],
"SubscriberInfo": [TxType.PERSISTENT],
"TermsVersion": [TxType.PERSISTENT],
"TrainingVersion": [TxType.PERSISTENT],
"UpdateAvailable": [TxType.CLEAR_ON_MANAGER_START],
"UpdateFailedCount": [TxType.CLEAR_ON_MANAGER_START],
"Version": [TxType.PERSISTENT],
"Offroad_ChargeDisabled": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"Offroad_ConnectivityNeeded": [TxType.CLEAR_ON_MANAGER_START],
"Offroad_ConnectivityNeededPrompt": [TxType.CLEAR_ON_MANAGER_START],
"Offroad_TemperatureTooHigh": [TxType.CLEAR_ON_MANAGER_START],
"Offroad_PandaFirmwareMismatch": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
"Offroad_InvalidTime": [TxType.CLEAR_ON_MANAGER_START],
"Offroad_IsTakingSnapshot": [TxType.CLEAR_ON_MANAGER_START],
"Offroad_NeosUpdate": [TxType.CLEAR_ON_MANAGER_START],
"Offroad_UpdateFailed": [TxType.CLEAR_ON_MANAGER_START],
}
def fsync_dir(path):
fd = os.open(path, os.O_RDONLY)
try:
os.fsync(fd)
finally:
os.close(fd)
class FileLock():
def __init__(self, path, create, lock_ex):
self._path = path
self._create = create
self._fd = None
self._lock_ex = lock_ex
def acquire(self):
self._fd = os.open(self._path, os.O_CREAT if self._create else 0)
fcntl.flock(self._fd, fcntl.LOCK_EX if self._lock_ex else fcntl.LOCK_SH)
def release(self):
if self._fd is not None:
os.close(self._fd)
self._fd = None
class DBAccessor():
def __init__(self, path):
self._path = path
self._vals = None
def keys(self):
self._check_entered()
return self._vals.keys()
def get(self, key):
self._check_entered()
if self._vals is None:
return None
try:
return self._vals[key]
except KeyError:
return None
def _get_lock(self, create, lock_ex):
lock = FileLock(os.path.join(self._path, ".lock"), create, lock_ex)
lock.acquire()
return lock
def _read_values_locked(self):
"""Callers should hold a lock while calling this method."""
vals = {}
try:
data_path = self._data_path()
keys = os.listdir(data_path)
for key in keys:
with open(os.path.join(data_path, key), "rb") as f:
vals[key] = f.read()
except (OSError, IOError) as e:
# Either the DB hasn't been created yet, or somebody wrote a bug and left the DB in an
# inconsistent state. Either way, return empty.
if e.errno == errno.ENOENT:
return {}
return vals
def _data_path(self):
return os.path.join(self._path, "d")
def _check_entered(self):
if self._vals is None:
raise Exception("Must call __enter__ before using DB")
class DBReader(DBAccessor):
def __enter__(self):
try:
lock = self._get_lock(False, False)
except OSError as e:
# Do not create lock if it does not exist.
if e.errno == errno.ENOENT:
self._vals = {}
return self
try:
# Read everything.
self._vals = self._read_values_locked()
return self
finally:
lock.release()
def __exit__(self, exc_type, exc_value, traceback):
pass
class DBWriter(DBAccessor):
def __init__(self, path):
super(DBWriter, self).__init__(path)
self._lock = None
self._prev_umask = None
def put(self, key, value):
self._vals[key] = value
def delete(self, key):
self._vals.pop(key, None)
def __enter__(self):
mkdirs_exists_ok(self._path)
# Make sure we can write and that permissions are correct.
self._prev_umask = os.umask(0)
try:
os.chmod(self._path, 0o777)
self._lock = self._get_lock(True, True)
self._vals = self._read_values_locked()
except Exception:
os.umask(self._prev_umask)
self._prev_umask = None
raise
return self
def __exit__(self, exc_type, exc_value, traceback):
self._check_entered()
try:
# data_path refers to the externally used path to the params. It is a symlink.
# old_data_path is the path currently pointed to by data_path.
# tempdir_path is a path where the new params will go, which the new data path will point to.
# new_data_path is a temporary symlink that will atomically overwrite data_path.
#
# The current situation is:
# data_path -> old_data_path
# We're going to write params data to tempdir_path
# tempdir_path -> params data
# Then point new_data_path to tempdir_path
# new_data_path -> tempdir_path
# Then atomically overwrite data_path with new_data_path
# data_path -> tempdir_path
old_data_path = None
new_data_path = None
tempdir_path = tempfile.mkdtemp(prefix=".tmp", dir=self._path)
try:
# Write back all keys.
os.chmod(tempdir_path, 0o777)
for k, v in self._vals.items():
with open(os.path.join(tempdir_path, k), "wb") as f:
f.write(v)
f.flush()
os.fsync(f.fileno())
fsync_dir(tempdir_path)
data_path = self._data_path()
try:
old_data_path = os.path.join(self._path, os.readlink(data_path))
except (OSError, IOError):
# NOTE(mgraczyk): If other DB implementations have bugs, this could cause
# copies to be left behind, but we still want to overwrite.
pass
new_data_path = "{}.link".format(tempdir_path)
os.symlink(os.path.basename(tempdir_path), new_data_path)
os.rename(new_data_path, data_path)
fsync_dir(self._path)
finally:
# If the rename worked, we can delete the old data. Otherwise delete the new one.
success = new_data_path is not None and os.path.exists(data_path) and (
os.readlink(data_path) == os.path.basename(tempdir_path))
if success:
if old_data_path is not None:
shutil.rmtree(old_data_path)
else:
shutil.rmtree(tempdir_path)
# Regardless of what happened above, there should be no link at new_data_path.
if new_data_path is not None and os.path.islink(new_data_path):
os.remove(new_data_path)
finally:
os.umask(self._prev_umask)
self._prev_umask = None
# Always release the lock.
self._lock.release()
self._lock = None
def read_db(params_path, key):
path = "%s/d/%s" % (params_path, key)
try:
with open(path, "rb") as f:
return f.read()
except IOError:
return None
def write_db(params_path, key, value):
if isinstance(value, str):
value = value.encode('utf8')
prev_umask = os.umask(0)
lock = FileLock(params_path + "/.lock", True, True)
lock.acquire()
try:
tmp_path = tempfile.NamedTemporaryFile(mode="wb", prefix=".tmp", dir=params_path, delete=False)
with tmp_path as f:
f.write(value)
f.flush()
os.fsync(f.fileno())
os.chmod(tmp_path.name, 0o666)
path = "%s/d/%s" % (params_path, key)
os.rename(tmp_path.name, path)
fsync_dir(os.path.dirname(path))
finally:
os.umask(prev_umask)
lock.release()
class Params():
def __init__(self, db=PARAMS):
self.db = db
# create the database if it doesn't exist...
if not os.path.exists(self.db + "/d"):
with self.transaction(write=True):
pass
def clear_all(self):
shutil.rmtree(self.db, ignore_errors=True)
with self.transaction(write=True):
pass
def transaction(self, write=False):
if write:
return DBWriter(self.db)
else:
return DBReader(self.db)
def _clear_keys_with_type(self, tx_type):
with self.transaction(write=True) as txn:
for key in keys:
if tx_type in keys[key]:
txn.delete(key)
def manager_start(self):
self._clear_keys_with_type(TxType.CLEAR_ON_MANAGER_START)
def panda_disconnect(self):
self._clear_keys_with_type(TxType.CLEAR_ON_PANDA_DISCONNECT)
def delete(self, key):
with self.transaction(write=True) as txn:
txn.delete(key)
def get(self, key, block=False, encoding=None):
if key not in keys:
raise UnknownKeyName(key)
while 1:
ret = read_db(self.db, key)
if not block or ret is not None:
break
# is polling really the best we can do?
time.sleep(0.05)
if ret is not None and encoding is not None:
ret = ret.decode(encoding)
return ret
def put(self, key, dat):
"""
Warning: This function blocks until the param is written to disk!
In very rare cases this can take over a second, and your code will hang.
Use the put_nonblocking helper function in time sensitive code, but
in general try to avoid writing params as much as possible.
"""
if key not in keys:
raise UnknownKeyName(key)
write_db(self.db, key, dat)
def put_nonblocking(key, val):
def f(key, val):
params = Params()
params.put(key, val)
t = threading.Thread(target=f, args=(key, val))
t.start()
return t
from common.params_pyx import Params, UnknownKeyName, put_nonblocking # pylint: disable=no-name-in-module, import-error
assert Params
assert UnknownKeyName
assert put_nonblocking

View File

@ -0,0 +1,16 @@
from libcpp.string cimport string
from libcpp cimport bool
cdef extern from "selfdrive/common/params.cc":
pass
cdef extern from "selfdrive/common/util.c":
pass
cdef extern from "selfdrive/common/params.h":
cdef cppclass Params:
Params(bool)
Params(string)
string get(string, bool) nogil
int delete_db_value(string)
int write_db_value(string, string)

View File

@ -0,0 +1,157 @@
# distutils: langauge = c++
import threading
import os
from libcpp cimport bool
from libcpp.string cimport string
from params_pxd cimport Params as c_Params
cdef enum TxType:
PERSISTENT = 1
CLEAR_ON_MANAGER_START = 2
CLEAR_ON_PANDA_DISCONNECT = 3
keys = {
b"AccessToken": [TxType.CLEAR_ON_MANAGER_START],
b"AthenadPid": [TxType.PERSISTENT],
b"CalibrationParams": [TxType.PERSISTENT],
b"CarBatteryCapacity": [TxType.PERSISTENT],
b"CarParams": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"CarParamsCache": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"CarVin": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"CommunityFeaturesToggle": [TxType.PERSISTENT],
b"CompletedTrainingVersion": [TxType.PERSISTENT],
b"DisablePowerDown": [TxType.PERSISTENT],
b"DisableUpdates": [TxType.PERSISTENT],
b"DoUninstall": [TxType.CLEAR_ON_MANAGER_START],
b"DongleId": [TxType.PERSISTENT],
b"GitBranch": [TxType.PERSISTENT],
b"GitCommit": [TxType.PERSISTENT],
b"GitRemote": [TxType.PERSISTENT],
b"GithubSshKeys": [TxType.PERSISTENT],
b"HasAcceptedTerms": [TxType.PERSISTENT],
b"HasCompletedSetup": [TxType.PERSISTENT],
b"IsDriverViewEnabled": [TxType.CLEAR_ON_MANAGER_START],
b"IsLdwEnabled": [TxType.PERSISTENT],
b"IsMetric": [TxType.PERSISTENT],
b"IsOffroad": [TxType.CLEAR_ON_MANAGER_START],
b"IsRHD": [TxType.PERSISTENT],
b"IsTakingSnapshot": [TxType.CLEAR_ON_MANAGER_START],
b"IsUpdateAvailable": [TxType.CLEAR_ON_MANAGER_START],
b"IsUploadRawEnabled": [TxType.PERSISTENT],
b"LastAthenaPingTime": [TxType.PERSISTENT],
b"LastUpdateTime": [TxType.PERSISTENT],
b"LastUpdateException": [TxType.PERSISTENT],
b"LiveParameters": [TxType.PERSISTENT],
b"OpenpilotEnabledToggle": [TxType.PERSISTENT],
b"LaneChangeEnabled": [TxType.PERSISTENT],
b"PandaFirmware": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"PandaFirmwareHex": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"PandaDongleId": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"Passive": [TxType.PERSISTENT],
b"RecordFront": [TxType.PERSISTENT],
b"ReleaseNotes": [TxType.PERSISTENT],
b"ShouldDoUpdate": [TxType.CLEAR_ON_MANAGER_START],
b"SubscriberInfo": [TxType.PERSISTENT],
b"TermsVersion": [TxType.PERSISTENT],
b"TrainingVersion": [TxType.PERSISTENT],
b"UpdateAvailable": [TxType.CLEAR_ON_MANAGER_START],
b"UpdateFailedCount": [TxType.CLEAR_ON_MANAGER_START],
b"Version": [TxType.PERSISTENT],
b"Offroad_ChargeDisabled": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"Offroad_ConnectivityNeeded": [TxType.CLEAR_ON_MANAGER_START],
b"Offroad_ConnectivityNeededPrompt": [TxType.CLEAR_ON_MANAGER_START],
b"Offroad_TemperatureTooHigh": [TxType.CLEAR_ON_MANAGER_START],
b"Offroad_PandaFirmwareMismatch": [TxType.CLEAR_ON_MANAGER_START, TxType.CLEAR_ON_PANDA_DISCONNECT],
b"Offroad_InvalidTime": [TxType.CLEAR_ON_MANAGER_START],
b"Offroad_IsTakingSnapshot": [TxType.CLEAR_ON_MANAGER_START],
b"Offroad_NeosUpdate": [TxType.CLEAR_ON_MANAGER_START],
b"Offroad_UpdateFailed": [TxType.CLEAR_ON_MANAGER_START],
}
def ensure_bytes(v):
if isinstance(v, str):
return v.encode()
else:
return v
class UnknownKeyName(Exception):
pass
cdef class Params:
cdef c_Params* p
def __cinit__(self, d=None, persistent_params=False):
if d is not None:
self.p = new c_Params(<string>d.encode())
else:
self.p = new c_Params(<bool>persistent_params)
def __dealloc__(self):
del self.p
def clear_all(self, tx_type=None):
for key in keys:
if tx_type is None or tx_type in keys[key]:
self.delete(key)
def manager_start(self):
self.clear_all(TxType.CLEAR_ON_MANAGER_START)
def panda_disconnect(self):
self.clear_all(TxType.CLEAR_ON_PANDA_DISCONNECT)
def get(self, key, block=False, encoding=None):
key = ensure_bytes(key)
if key not in keys:
raise UnknownKeyName(key)
cdef string k = key
cdef bool b = block
cdef string val
with nogil:
val = self.p.get(k, b)
if val == b"":
if block:
# If we got no value while running in blocked mode
# it means we got an interrupt while waiting
raise KeyboardInterrupt
else:
return None
if encoding is not None:
return val.decode(encoding)
else:
return val
def put(self, key, dat):
"""
Warning: This function blocks until the param is written to disk!
In very rare cases this can take over a second, and your code will hang.
Use the put_nonblocking helper function in time sensitive code, but
in general try to avoid writing params as much as possible.
"""
key = ensure_bytes(key)
dat = ensure_bytes(dat)
if key not in keys:
raise UnknownKeyName(key)
self.p.write_db_value(key, dat)
def delete(self, key):
key = ensure_bytes(key)
self.p.delete_db_value(key)
def put_nonblocking(key, val, d=None):
def f(key, val):
params = Params(d)
params.put(key, val)
t = threading.Thread(target=f, args=(key, val))
t.start()
return t

View File

@ -0,0 +1,21 @@
import os
from distutils.core import Extension, setup
from Cython.Build import cythonize
from common.cython_hacks import BuildExtWithoutPlatformSuffix
from common.basedir import BASEDIR
sourcefiles = ['params_pyx.pyx']
extra_compile_args = ["-std=c++11"]
setup(name='common',
cmdclass={'build_ext': BuildExtWithoutPlatformSuffix},
ext_modules=cythonize(
Extension(
"params_pyx",
language="c++",
sources=sourcefiles,
include_dirs=[BASEDIR, os.path.join(BASEDIR, 'selfdrive')],
extra_compile_args=extra_compile_args
)
)
)

View File

@ -6,7 +6,7 @@ import shutil
import stat
import unittest
from common.params import Params, UnknownKeyName
from common.params import Params, UnknownKeyName, put_nonblocking
class TestParams(unittest.TestCase):
def setUp(self):
@ -67,5 +67,20 @@ class TestParams(unittest.TestCase):
st_mode = os.stat(f"{self.tmpdir}/d/DongleId").st_mode
assert (st_mode & permissions) == permissions
def test_delete_not_there(self):
assert self.params.get("CarParams") is None
self.params.delete("CarParams")
assert self.params.get("CarParams") is None
def test_put_non_blocking_with_get_block(self):
q = Params(self.tmpdir)
def _delayed_writer():
time.sleep(0.1)
put_nonblocking("CarParams", "test", self.tmpdir)
threading.Thread(target=_delayed_writer).start()
assert q.get("CarParams") is None
assert q.get("CarParams", True) == b"test"
if __name__ == "__main__":
unittest.main()

View File

@ -30,6 +30,9 @@ common/file_helpers.py
common/logging_extra.py
common/numpy_fast.py
common/params.py
common/params_pxd.pxd
common/params_pyx.pyx
common/params_pyx_setup.py
common/xattr.py
common/profiler.py
common/basedir.py

View File

@ -71,7 +71,7 @@ void safety_setter_thread() {
return;
};
std::vector<char> value_vin = read_db_bytes("CarVin");
std::vector<char> value_vin = Params().read_db_bytes("CarVin");
if (value_vin.size() > 0) {
// sanity check VIN format
assert(value_vin.size() == 17);
@ -93,7 +93,7 @@ void safety_setter_thread() {
return;
};
params = read_db_bytes("CarParams");
params = Params().read_db_bytes("CarParams");
if (params.size() > 0) break;
usleep(100*1000);
}
@ -124,13 +124,15 @@ bool usb_connect() {
return false;
}
Params params = Params();
if (getenv("BOARDD_LOOPBACK")) {
panda->set_loopback(true);
}
const char *fw_sig_buf = panda->get_firmware_version();
if (fw_sig_buf){
write_db_value("PandaFirmware", fw_sig_buf, 128);
params.write_db_value("PandaFirmware", fw_sig_buf, 128);
// Convert to hex for offroad
char fw_sig_hex_buf[16] = {0};
@ -139,7 +141,7 @@ bool usb_connect() {
fw_sig_hex_buf[2*i+1] = NIBBLE_TO_HEX((uint8_t)fw_sig_buf[i] & 0xF);
}
write_db_value("PandaFirmwareHex", fw_sig_hex_buf, 16);
params.write_db_value("PandaFirmwareHex", fw_sig_hex_buf, 16);
LOGW("fw signature: %.*s", 16, fw_sig_hex_buf);
delete[] fw_sig_buf;
@ -150,7 +152,7 @@ bool usb_connect() {
if (serial_buf) {
size_t serial_sz = strnlen(serial_buf, 16);
write_db_value("PandaDongleId", serial_buf, serial_sz);
params.write_db_value("PandaDongleId", serial_buf, serial_sz);
LOGW("panda serial: %.*s", serial_sz, serial_buf);
delete[] serial_buf;
@ -269,6 +271,7 @@ void can_health_thread() {
uint32_t no_ignition_cnt = 0;
bool ignition_last = false;
Params params = Params();
// Broadcast empty health message when panda is not yet connected
while (!panda){
@ -318,9 +321,9 @@ void can_health_thread() {
// clear VIN, CarParams, and set new safety on car start
if (ignition && !ignition_last) {
int result = delete_db_value("CarVin");
int result = params.delete_db_value("CarVin");
assert((result == 0) || (result == ERR_NO_VALUE));
result = delete_db_value("CarParams");
result = params.delete_db_value("CarParams");
assert((result == 0) || (result == ERR_NO_VALUE));
if (!safety_setter_thread_running) {

View File

@ -183,7 +183,7 @@ struct VisionState {
void* frontview_thread(void *arg) {
int err;
VisionState *s = (VisionState*)arg;
s->rhd_front = read_db_bool("IsRHD");
s->rhd_front = Params().read_db_bool("IsRHD");
set_thread_name("frontview");
err = set_realtime_priority(51);
@ -1538,7 +1538,7 @@ void free_buffers(VisionState *s) {
clReleaseProgram(s->prg_rgb_laplacian);
clReleaseKernel(s->krnl_rgb_laplacian);
}
void party(VisionState *s) {

View File

@ -13,6 +13,8 @@
#include <map>
#include <string>
#include <iostream>
#include <csignal>
#include <string.h>
#include "common/util.h"
@ -37,9 +39,15 @@ static const char* persistent_params_path = default_params_path;
} //namespace
volatile sig_atomic_t params_do_exit = 0;
void params_sig_handler(int signal) {
std::cout << "got signal" << std::endl;
params_do_exit = 1;
}
static int fsync_dir(const char* path){
int result = 0;
int fd = open(path, O_RDONLY);
int fd = open(path, O_RDONLY, 0755);
if (fd < 0){
result = -1;
@ -64,15 +72,44 @@ static int fsync_dir(const char* path){
}
}
static int ensure_dir_exists(const char* path) {
struct stat st;
if (stat(path, &st) == -1) {
return mkdir(path, 0700);
static int mkdir_p(std::string path) {
char * _path = (char *)path.c_str();
for (char *p = _path + 1; *p; p++) {
if (*p == '/') {
*p = '\0'; // Temporarily truncate
if (mkdir(_path, 0775) != 0) {
if (errno != EEXIST) return -1;
}
*p = '/';
}
}
if (mkdir(_path, 0775) != 0) {
if (errno != EEXIST) return -1;
}
return 0;
}
int write_db_value(const char* key, const char* value, size_t value_size, bool persistent_param) {
static int ensure_dir_exists(std::string path) {
// TODO: replace by std::filesystem::create_directories
return mkdir_p(path.c_str());
}
Params::Params(bool persistent_param){
const char * path = persistent_param ? persistent_params_path : default_params_path;
params_path = std::string(path);
}
Params::Params(std::string path) {
params_path = path;
}
int Params::write_db_value(std::string key, std::string dat){
return write_db_value(key.c_str(), dat.c_str(), dat.length());
}
int Params::write_db_value(const char* key, const char* value, size_t value_size) {
// Information about safely and atomically writing a file: https://lwn.net/Articles/457667/
// 1) Create temp file
// 2) Write data to temp file
@ -83,11 +120,9 @@ int write_db_value(const char* key, const char* value, size_t value_size, bool p
int lock_fd = -1;
int tmp_fd = -1;
int result;
char tmp_path[1024];
char path[1024];
char *tmp_dir;
std::string path;
std::string tmp_path;
ssize_t bytes_written;
const char* params_path = persistent_param ? persistent_params_path : default_params_path;
// Make sure params path exists
result = ensure_dir_exists(params_path);
@ -95,59 +130,43 @@ int write_db_value(const char* key, const char* value, size_t value_size, bool p
goto cleanup;
}
result = snprintf(path, sizeof(path), "%s/d", params_path);
if (result < 0) {
goto cleanup;
}
// See if the symlink exists, otherwise create it
path = params_path + "/d";
struct stat st;
if (stat(path, &st) == -1) {
if (stat(path.c_str(), &st) == -1) {
// Create temp folder
result = snprintf(path, sizeof(path), "%s/.tmp_XXXXXX", params_path);
if (result < 0) {
goto cleanup;
}
tmp_dir = mkdtemp(path);
if (tmp_dir == NULL){
path = params_path + "/.tmp_XXXXXX";
char *t = mkdtemp((char*)path.c_str());
if (t == NULL){
goto cleanup;
}
std::string tmp_dir(t);
// Set permissions
result = chmod(tmp_dir, 0777);
result = chmod(tmp_dir.c_str(), 0777);
if (result < 0) {
goto cleanup;
}
// Symlink it to temp link
result = snprintf(tmp_path, sizeof(tmp_path), "%s.link", tmp_dir);
if (result < 0) {
goto cleanup;
}
result = symlink(tmp_dir, tmp_path);
tmp_path = tmp_dir + ".link";
result = symlink(tmp_dir.c_str(), tmp_path.c_str());
if (result < 0) {
goto cleanup;
}
// Move symlink to <params>/d
result = snprintf(path, sizeof(path), "%s/d", params_path);
if (result < 0) {
goto cleanup;
}
result = rename(tmp_path, path);
path = params_path + "/d";
result = rename(tmp_path.c_str(), path.c_str());
if (result < 0) {
goto cleanup;
}
}
// Write value to temp.
result =
snprintf(tmp_path, sizeof(tmp_path), "%s/.tmp_value_XXXXXX", params_path);
if (result < 0) {
goto cleanup;
}
tmp_fd = mkstemp(tmp_path);
tmp_path = params_path + "/.tmp_value_XXXXXX";
tmp_fd = mkstemp((char*)tmp_path.c_str());
bytes_written = write(tmp_fd, value, value_size);
if (bytes_written != value_size) {
result = -20;
@ -155,17 +174,11 @@ int write_db_value(const char* key, const char* value, size_t value_size, bool p
}
// Build lock path
result = snprintf(path, sizeof(path), "%s/.lock", params_path);
if (result < 0) {
goto cleanup;
}
lock_fd = open(path, O_CREAT);
path = params_path + "/.lock";
lock_fd = open(path.c_str(), O_CREAT, 0775);
// Build key path
result = snprintf(path, sizeof(path), "%s/d/%s", params_path, key);
if (result < 0) {
goto cleanup;
}
path = params_path + "/d/" + std::string(key);
// Take lock.
result = flock(lock_fd, LOCK_EX);
@ -186,18 +199,14 @@ int write_db_value(const char* key, const char* value, size_t value_size, bool p
}
// Move temp into place.
result = rename(tmp_path, path);
result = rename(tmp_path.c_str(), path.c_str());
if (result < 0) {
goto cleanup;
}
// fsync parent directory
result = snprintf(path, sizeof(path), "%s/d", params_path);
if (result < 0) {
goto cleanup;
}
result = fsync_dir(path);
path = params_path + "/d";
result = fsync_dir(path.c_str());
if (result < 0) {
goto cleanup;
}
@ -209,25 +218,25 @@ cleanup:
}
if (tmp_fd >= 0) {
if (result < 0) {
remove(tmp_path);
remove(tmp_path.c_str());
}
close(tmp_fd);
}
return result;
}
int delete_db_value(const char* key, bool persistent_param) {
int Params::delete_db_value(std::string key) {
return delete_db_value(key.c_str());
}
int Params::delete_db_value(const char* key) {
int lock_fd = -1;
int result;
char path[1024];
const char* params_path = persistent_param ? persistent_params_path : default_params_path;
std::string path;
// Build lock path, and open lockfile
result = snprintf(path, sizeof(path), "%s/.lock", params_path);
if (result < 0) {
goto cleanup;
}
lock_fd = open(path, O_CREAT);
path = params_path + "/.lock";
lock_fd = open(path.c_str(), O_CREAT, 0775);
// Take lock.
result = flock(lock_fd, LOCK_EX);
@ -235,26 +244,17 @@ int delete_db_value(const char* key, bool persistent_param) {
goto cleanup;
}
// Build key path
result = snprintf(path, sizeof(path), "%s/d/%s", params_path, key);
if (result < 0) {
goto cleanup;
}
// Delete value.
result = remove(path);
path = params_path + "/d/" + std::string(key);
result = remove(path.c_str());
if (result != 0) {
result = ERR_NO_VALUE;
goto cleanup;
}
// fsync parent directory
result = snprintf(path, sizeof(path), "%s/d", params_path);
if (result < 0) {
goto cleanup;
}
result = fsync_dir(path);
path = params_path + "/d";
result = fsync_dir(path.c_str());
if (result < 0) {
goto cleanup;
}
@ -267,39 +267,58 @@ cleanup:
return result;
}
int read_db_value(const char* key, char** value, size_t* value_sz, bool persistent_param) {
char path[1024];
const char* params_path = persistent_param ? persistent_params_path : default_params_path;
std::string Params::get(std::string key, bool block){
char* value;
size_t size;
int r;
int result = snprintf(path, sizeof(path), "%s/d/%s", params_path, key);
if (result < 0) {
return result;
if (block){
r = read_db_value_blocking((const char*)key.c_str(), &value, &size);
} else {
r = read_db_value((const char*)key.c_str(), &value, &size);
}
*value = static_cast<char*>(read_file(path, value_sz));
if (r == 0){
std::string s(value, size);
free(value);
return s;
} else {
return "";
}
}
int Params::read_db_value(const char* key, char** value, size_t* value_sz) {
std::string path = params_path + "/d/" + std::string(key);
*value = static_cast<char*>(read_file(path.c_str(), value_sz));
if (*value == NULL) {
return -22;
}
return 0;
}
void read_db_value_blocking(const char* key, char** value, size_t* value_sz, bool persistent_param) {
while (1) {
const int result = read_db_value(key, value, value_sz, persistent_param);
int Params::read_db_value_blocking(const char* key, char** value, size_t* value_sz) {
params_do_exit = 0;
void (*prev_handler_sigint)(int) = std::signal(SIGINT, params_sig_handler);
void (*prev_handler_sigterm)(int) = std::signal(SIGTERM, params_sig_handler);
while (!params_do_exit) {
const int result = read_db_value(key, value, value_sz);
if (result == 0) {
return;
break;
} else {
// Sleep for 0.1 seconds.
usleep(100000);
usleep(100000); // 0.1 s
}
}
std::signal(SIGINT, prev_handler_sigint);
std::signal(SIGTERM, prev_handler_sigterm);
return params_do_exit; // Return 0 if we had no interrupt
}
int read_db_all(std::map<std::string, std::string> *params, bool persistent_param) {
int Params::read_db_all(std::map<std::string, std::string> *params) {
int err = 0;
const char* params_path = persistent_param ? persistent_params_path : default_params_path;
std::string lock_path = util::string_format("%s/.lock", params_path);
std::string lock_path = params_path + "/.lock";
int lock_fd = open(lock_path.c_str(), 0);
if (lock_fd < 0) return -1;
@ -310,7 +329,7 @@ int read_db_all(std::map<std::string, std::string> *params, bool persistent_para
return err;
}
std::string key_path = util::string_format("%s/d", params_path);
std::string key_path = params_path + "/d";
DIR *d = opendir(key_path.c_str());
if (!d) {
close(lock_fd);
@ -321,7 +340,7 @@ int read_db_all(std::map<std::string, std::string> *params, bool persistent_para
while ((de = readdir(d))) {
if (!isalnum(de->d_name[0])) continue;
std::string key = std::string(de->d_name);
std::string value = util::read_file(util::string_format("%s/%s", key_path.c_str(), key.c_str()));
std::string value = util::read_file(key_path + "/" + key);
(*params)[key] = value;
}
@ -332,11 +351,11 @@ int read_db_all(std::map<std::string, std::string> *params, bool persistent_para
return 0;
}
std::vector<char> read_db_bytes(const char* param_name, bool persistent_param) {
std::vector<char> Params::read_db_bytes(const char* param_name) {
std::vector<char> bytes;
char* value;
size_t sz;
int result = read_db_value(param_name, &value, &sz, persistent_param);
int result = read_db_value(param_name, &value, &sz);
if (result == 0) {
bytes.assign(value, value+sz);
free(value);
@ -344,7 +363,7 @@ std::vector<char> read_db_bytes(const char* param_name, bool persistent_param) {
return bytes;
}
bool read_db_bool(const char* param_name, bool persistent_param) {
std::vector<char> bytes = read_db_bytes(param_name, persistent_param);
bool Params::read_db_bool(const char* param_name) {
std::vector<char> bytes = read_db_bytes(param_name);
return bytes.size() > 0 and bytes[0] == '1';
}

View File

@ -1,44 +1,47 @@
#pragma once
#include <stddef.h>
#ifdef __cplusplus
extern "C" {
#endif
#define ERR_NO_VALUE -33
int write_db_value(const char* key, const char* value, size_t value_size, bool persistent_param = false);
// Reads a value from the params database.
// Inputs:
// key: The key to read.
// value: A pointer where a newly allocated string containing the db value will
// be written.
// value_sz: A pointer where the size of value will be written. Does not
// include the NULL terminator.
// persistent_param: Boolean indicating if the param store in the /persist partition is to be used.
// e.g. for sensor calibration files. Will not be cleared after wipe or re-install.
//
// Returns: Negative on failure, otherwise 0.
int read_db_value(const char* key, char** value, size_t* value_sz, bool persistent_param = false);
// Delete a value from the params database.
// Inputs are the same as read_db_value, without value and value_sz.
int delete_db_value(const char* key, bool persistent_param = false);
// Reads a value from the params database, blocking until successful.
// Inputs are the same as read_db_value.
void read_db_value_blocking(const char* key, char** value, size_t* value_sz, bool persistent_param = false);
#ifdef __cplusplus
} // extern "C"
#endif
#ifdef __cplusplus
#include <map>
#include <string>
#include <vector>
int read_db_all(std::map<std::string, std::string> *params, bool persistent_param = false);
std::vector<char> read_db_bytes(const char* param_name, bool persistent_param = false);
bool read_db_bool(const char* param_name, bool persistent_param = false);
#endif
#define ERR_NO_VALUE -33
class Params {
private:
std::string params_path;
public:
Params(bool persistent_param = false);
Params(std::string path);
int write_db_value(std::string key, std::string dat);
int write_db_value(const char* key, const char* value, size_t value_size);
// Reads a value from the params database.
// Inputs:
// key: The key to read.
// value: A pointer where a newly allocated string containing the db value will
// be written.
// value_sz: A pointer where the size of value will be written. Does not
// include the NULL terminator.
// persistent_param: Boolean indicating if the param store in the /persist partition is to be used.
// e.g. for sensor calibration files. Will not be cleared after wipe or re-install.
//
// Returns: Negative on failure, otherwise 0.
int read_db_value(const char* key, char** value, size_t* value_sz);
// Delete a value from the params database.
// Inputs are the same as read_db_value, without value and value_sz.
int delete_db_value(std::string key);
int delete_db_value(const char* key);
// Reads a value from the params database, blocking until successful.
// Inputs are the same as read_db_value.
int read_db_value_blocking(const char* key, char** value, size_t* value_sz);
int read_db_all(std::map<std::string, std::string> *params);
std::vector<char> read_db_bytes(const char* param_name);
bool read_db_bool(const char* param_name);
std::string get(std::string key, bool block=false);
};

View File

@ -522,30 +522,31 @@ kj::Array<capnp::word> gen_init_data() {
if (!clean) {
init.setDirty(true);
}
Params params = Params();
std::vector<char> git_commit = read_db_bytes("GitCommit");
std::vector<char> git_commit = params.read_db_bytes("GitCommit");
if (git_commit.size() > 0) {
init.setGitCommit(capnp::Text::Reader(git_commit.data(), git_commit.size()));
}
std::vector<char> git_branch = read_db_bytes("GitBranch");
std::vector<char> git_branch = params.read_db_bytes("GitBranch");
if (git_branch.size() > 0) {
init.setGitBranch(capnp::Text::Reader(git_branch.data(), git_branch.size()));
}
std::vector<char> git_remote = read_db_bytes("GitRemote");
std::vector<char> git_remote = params.read_db_bytes("GitRemote");
if (git_remote.size() > 0) {
init.setGitRemote(capnp::Text::Reader(git_remote.data(), git_remote.size()));
}
init.setPassive(read_db_bool("Passive"));
init.setPassive(params.read_db_bool("Passive"));
{
// log params
std::map<std::string, std::string> params;
read_db_all(&params);
auto lparams = init.initParams().initEntries(params.size());
std::map<std::string, std::string> params_map;
params.read_db_all(&params_map);
auto lparams = init.initParams().initEntries(params_map.size());
int i = 0;
for (auto& kv : params) {
for (auto& kv : params_map) {
auto lentry = lparams[i];
lentry.setKey(kv.first);
lentry.setValue(kv.second);
@ -613,7 +614,7 @@ int main(int argc, char** argv) {
}
bool record_front = true;
#ifndef QCOM2
record_front = read_db_bool("RecordFront");
record_front = Params().read_db_bool("RecordFront");
#endif
setpriority(PRIO_PROCESS, 0, -12);

View File

@ -18,6 +18,7 @@ from common.hardware import HARDWARE, ANDROID, PC
WEBCAM = os.getenv("WEBCAM") is not None
sys.path.append(os.path.join(BASEDIR, "pyextra"))
os.environ['BASEDIR'] = BASEDIR
os.environ['PARAMS_PATH'] = PARAMS
TOTAL_SCONS_NODES = 1005
prebuilt = os.path.exists(os.path.join(BASEDIR, 'prebuilt'))
@ -536,8 +537,6 @@ def uninstall():
HARDWARE.reboot(reason="recovery")
def main():
os.environ['PARAMS_PATH'] = PARAMS
if ANDROID:
# the flippening!
os.system('LD_LIBRARY_PATH="" content insert --uri content://settings/system --bind name:s:user_rotation --bind value:i:1')

View File

@ -28,7 +28,7 @@ void dmonitoring_init(DMonitoringModelState* s) {
int runtime = USE_DSP_RUNTIME;
#endif
s->m = new DefaultRunModel(model_path, (float*)&s->output, OUTPUT_SIZE, runtime);
s->is_rhd = read_db_bool("IsRHD");
s->is_rhd = Params().read_db_bool("IsRHD");
}
template <class T>

View File

@ -54,7 +54,7 @@ void model_init(ModelState* s, cl_device_id device_id, cl_context context, int t
s->traffic_convention = std::make_unique<float[]>(TRAFFIC_CONVENTION_LEN);
s->m->addTrafficConvention(s->traffic_convention.get(), TRAFFIC_CONVENTION_LEN);
bool is_rhd = read_db_bool("IsRHD");
bool is_rhd = Params().read_db_bool("IsRHD");
if (is_rhd) {
s->traffic_convention[1] = 1.0;
} else {

View File

@ -5,11 +5,11 @@ from nose.tools import nottest
from common.hardware import PC
from common.apk import update_apks, start_offroad, pm_apply_packages, android_packages
from common.params import Params
from selfdrive.version import training_version, terms_version
from selfdrive.manager import start_managed_process, kill_managed_process, get_running
def set_params_enabled():
from common.params import Params
params = Params()
params.put("HasAcceptedTerms", terms_version)
params.put("HasCompletedSetup", "1")

View File

@ -79,7 +79,7 @@ static void handle_vision_touch(UIState *s, int touch_x, int touch_y) {
if (!s->scene.frontview) {
s->scene.uilayout_sidebarcollapsed = !s->scene.uilayout_sidebarcollapsed;
} else {
write_db_value("IsDriverViewEnabled", "0", 1);
Params().write_db_value("IsDriverViewEnabled", "0", 1);
}
}
}

View File

@ -43,7 +43,7 @@ ParamsToggle::ParamsToggle(QString param, QString title, QString description, QS
setLayout(hlayout);
checkbox->setChecked(read_db_bool(param.toStdString().c_str()));
checkbox->setChecked(Params().read_db_bool(param.toStdString().c_str()));
setStyleSheet(R"(
QCheckBox { font-size: 70px }
@ -58,7 +58,7 @@ ParamsToggle::ParamsToggle(QString param, QString title, QString description, QS
void ParamsToggle::checkboxClicked(int state){
char value = state ? '1': '0';
write_db_value(param.toStdString().c_str(), &value, 1);
Params().write_db_value(param.toStdString().c_str(), &value, 1);
}
SettingsWindow::SettingsWindow(QWidget *parent) : QWidget(parent) {

View File

@ -19,7 +19,7 @@ extern volatile sig_atomic_t do_exit;
int write_param_float(float param, const char* param_name, bool persistent_param) {
char s[16];
int size = snprintf(s, sizeof(s), "%f", param);
return write_db_value(param_name, s, size < sizeof(s) ? size : sizeof(s), persistent_param);
return Params(persistent_param).write_db_value(param_name, s, size < sizeof(s) ? size : sizeof(s));
}
void ui_init(UIState *s) {

View File

@ -204,7 +204,7 @@ int read_param(T* param, const char *param_name, bool persistent_param = false){
char *value;
size_t sz;
int result = read_db_value(param_name, &value, &sz, persistent_param);
int result = Params(persistent_param).read_db_value(param_name, &value, &sz);
if (result == 0){
std::string s = std::string(value, sz); // value is not null terminated
free(value);