Compare commits

...

10 Commits

Author SHA1 Message Date
Vladimir Oltean f078f19339 ts2phc_phc_master: make use of new kernel API for perout waveform
This API was introduced for 2 reasons:

1. Some hardware can emit PPS signals but not starting from arbitrary
   absolute times, but rather phase-aligned to the beginning of a
   second. We _could_ patch ts2phc to always specify a start time of
   0.000000000 to PTP_PEROUT_REQUEST, but in practice, we would never
   know whether that would actually work with all in-tree PHC drivers.
   So there was a need for a new flag that only specifies the phase of
   the periodic signal, and not the absolute start time.

2. Some hardware can, rather unfortunately, not distinguish between a
   rising and a falling extts edge. And, since whatever rises also has
   to fall before rising again, the strategy in ts2phc is to set a
   'large' pulse width (half the period) and ignore the extts event
   corresponding to the mid-way between one second and another. This is
   all fine, but currently, ts2phc.pulsewidth is a read-only property in
   the config file. The kernel is not instructed in any way to use this
   value, it is simply that must be configured based on prior knowledge
   of the PHC's implementation. This API changes that.

The introduction of a phase adjustment for the master PHC means we have
to adjust our approximation of the precise perout timestamp. We put that
code into a common function and convert all call sites to call that. We
also need to do the same thing for the edge ignoring logic.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:49 +03:00
Vladimir Oltean d5b3b1e07b ts2phc: allow the PHC PPS master to be synchronized
Now that we are registering a clock even for the PPS master when it
supports that (i.e. when it is a PHC), introduce a new API to retrieve
its clock in order to add timestamps to it.

The PHC master can be synchronized to the extts events of a PHC slave,
when in automatic mode.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:49 +03:00
Vladimir Oltean 3045a49855 ts2phc: reconfigure sync direction by subscribing to ptp4l port events
Monitor the port state change events from ptp4l, and use that
information to determine the "source" clock.

Then synchronize all other clocks in our list to that source, by feeding
into their respective servo loop an offset equal to the delta between
their timestamp and the timestamp of the source clock. All timestamps
are representative of the same event, which is the most recent perout
pulse of the ts2phc master.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:49 +03:00
Vladimir Oltean a6c19b7154 ts2phc: split slave poll from servo loop
Since it has been argued that:

- a ts2phc slave deals with extts events
- a clock deals with synchronization via a servo loop

then the code for synchronization should not be part of the
implementation of a ts2phc slave. Move it to the main ts2phc.c.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:49 +03:00
Vladimir Oltean 1441562149 ts2phc_slave: print master offset
Make this information more visible by default, since it is the key
output of this program.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
2020-08-30 02:56:49 +03:00
Vladimir Oltean 314ff98903 ts2phc: instantiate a pmc node
This introduces the '-a' option in ts2phc, an option inspired from
phc2sys that puts the clocks in "automatic" mode. In this mode, ts2phc
listens, as a PMC, to port state change events from ptp4l, and detects
which port state machine, if any, has transitioned to PS_SLAVE. That
port's clock will become the synchronization master for the hierarchy
described by ts2phc.

The use case is a multi-switch DSA setup with boundary_clock_jbod, where
there is only one grandmaster, connected to one switch's port. The other
switches, connected together through a PPS signal, must adapt themselves
to this new source of time, while the switch connected to the GM must
not be synchronized by ts2phc because it is already synchronized by
ptp4l.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:48 +03:00
Vladimir Oltean 3f2e73e91b ts2phc: instantiate a full clock structure for every PHC master
This propagates the use of "struct ts2phc_private" all the way into the
master API, in preparation of a new use case that will be supported
soon: some PPS masters (to be precise, the "PHC" kind) instantiate a
struct clock which could be disciplined by ts2phc.

When a PHC A emits a pulse and another PHC B timestamps it, the offset
between their precise timestamps can be used to synchronize either one
of them. So far in ts2phc, only the slave PHC (the one using extts) has
been synchronized to the master (the one using perout).

This is partly because there is no proper kernel API to report the
precise timestamp of a perout pulse. We only have the periodic API, and
that doesn't report precise timestamps either; we just use vague
approximations of what the PPS master PHC's time was, based on reading
that PHC immediately after a slave extts event was received by the
application. While this is far from ideal, it does work, and does allow
PHC A to be synchronized to B.

This is particularly useful with the yet-to-be-introduced "automatic"
mode of ts2phc (similar to '-a' of phc2sys), and the PPS distribution
tree is fixed in hardware (as opposed to port states, which in
"automatic" mode are dynamic, as the name suggests).

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:13 +03:00
Vladimir Oltean 1967a5fc26 ts2phc: instantiate a full clock structure for every slave PHC
Slaves in ts2phc are PHC devices that implement the extts kernel API.
They are slaves just in the sense that they timestamp a pulse emitted by
somebody else.

Currently in ts2phc, PPS slaves are also the only candidates for the
clocks that get synchronized. There are 2 aspects that make this too
restrictive:

- Not all PPS slaves may be synchronization slaves. Consider a dynamic
  environment of multiple DSA switches using boundary_clock_jbod, and
  only one port is in the PS_SLAVE state. In that case, the clock of
  that port should be the synchronization master (called the "source
  clock" from now on, as far as ts2phc is concerned), regardless of
  whether it supports the extts API or not.

- Not all synchronization slaves may be PPS slaves. Specifically, the
  "PHC" type of PPS master in ts2phc can also be, fundamentally,
  disciplined. The code should be prepared to handle this by recognizing
  that things that can be disciplined by a servo should be represented
  by a "struct clock", and things that can timestamp external events
  should be represented by a "struct ts2phc_slave".

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
2020-08-30 02:56:10 +03:00
Vladimir Oltean 703c3c7bc7 ts2phc: create a private data structure
Eliminate the ad-hoc use of global variables in the ts2phc program by
introducing one data structure that incorporates them. This might make
the code more understandable to people coming from a kernel background,
since it resembles the type of data organization used there. It is also
now closer to the data organization of phc2sys, a similar program in
both purpose and implementation.

The reason why this is needed has to do with the ts2phc polymorphism for
a PPS master.

In the next patches, PPS masters will expose a struct clock, which will
be synchronized from the main ts2phc.c.

Not all PPS masters will expose a clock, only the PHC kind will. So the
current object encapsulation model needs to be loosened up little bit,
because the main ts2phc.c needs to synchronize a list of clocks, list
which is populated by the slaves and the masters which are capable of
being synchronized.

So instead of having the translation modules of ts2phc communicate
through global variables, let's make struct ts2phc_private the common
working space for the entire program, which is a paradigm that is more
natural.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
2020-08-30 02:28:04 +03:00
Vladimir Oltean f44b3d8b6c phc2sys: break out pmc code into pmc_common.c
The code through which phc2sys sends various PTP management messages to
ptp4l via pmc can be reused.

This patch is a trivial movement of that code to a separate translation
module, outside of phc2sys. This makes it available to other programs
that want to subscribe to port state change events too, such as ts2phc.

Signed-off-by: Vladimir Oltean <olteanv@gmail.com>
Reviewed-by: Jacob Keller <jacob.e.keller@intel.com>
2020-08-30 01:48:48 +03:00
20 changed files with 1391 additions and 622 deletions

View File

@ -314,6 +314,7 @@ struct config_item config_tab[] = {
GLOB_ITEM_STR("ts2phc.nmea_remote_host", ""),
GLOB_ITEM_STR("ts2phc.nmea_remote_port", ""),
GLOB_ITEM_STR("ts2phc.nmea_serialport", "/dev/ttyS0"),
PORT_ITEM_INT("ts2phc.perout_phase", -1, 0, 999999999),
PORT_ITEM_INT("ts2phc.pin_index", 0, 0, INT_MAX),
GLOB_ITEM_INT("ts2phc.pulsewidth", 500000000, 1000000, 999000000),
PORT_ITEM_ENU("tsproc_mode", TSPROC_FILTER, tsproc_enu),

View File

@ -27,7 +27,8 @@ FILTERS = filter.o mave.o mmedian.o
SERVOS = linreg.o ntpshm.o nullf.o pi.o servo.o
TRANSP = raw.o transport.o udp.o udp6.o uds.o
TS2PHC = ts2phc.o lstab.o nmea.o serial.o sock.o ts2phc_generic_master.o \
ts2phc_master.o ts2phc_phc_master.o ts2phc_nmea_master.o ts2phc_slave.o
ts2phc_master.o ts2phc_phc_master.o ts2phc_nmea_master.o ts2phc_slave.o \
pmc_common.o transport.o msg.o tlv.o uds.o udp.o udp6.o raw.o
OBJ = bmc.o clock.o clockadj.o clockcheck.o config.o designated_fsm.o \
e2e_tc.o fault.o $(FILTERS) fsm.o hash.o interface.o monitor.o msg.o phc.o \
port.o port_signaling.o pqueue.o print.o ptp4l.o p2p_tc.o rtnl.o $(SERVOS) \

View File

@ -97,6 +97,59 @@ struct compat_ptp_clock_caps {
#endif /*LINUX_VERSION_CODE < 5.8*/
/*
* Bits of the ptp_perout_request.flags field:
*/
#ifndef PTP_PEROUT_ONE_SHOT
#define PTP_PEROUT_ONE_SHOT (1<<0)
#endif
#ifndef PTP_PEROUT_DUTY_CYCLE
#define PTP_PEROUT_DUTY_CYCLE (1<<1)
#endif
#ifndef PTP_PEROUT_PHASE
#define PTP_PEROUT_PHASE (1<<2)
#endif
#if LINUX_VERSION_CODE < KERNEL_VERSION(5,9,0)
/* from upcoming Linux kernel version 5.9 */
struct compat_ptp_perout_request {
union {
/*
* Absolute start time.
* Valid only if (flags & PTP_PEROUT_PHASE) is unset.
*/
struct ptp_clock_time start;
/*
* Phase offset. The signal should start toggling at an
* unspecified integer multiple of the period, plus this value.
* The start time should be "as soon as possible".
* Valid only if (flags & PTP_PEROUT_PHASE) is set.
*/
struct ptp_clock_time phase;
};
struct ptp_clock_time period; /* Desired period, zero means disable. */
unsigned int index; /* Which channel to configure. */
unsigned int flags;
union {
/*
* The "on" time of the signal.
* Must be lower than the period.
* Valid only if (flags & PTP_PEROUT_DUTY_CYCLE) is set.
*/
struct ptp_clock_time on;
/* Reserved for future use. */
unsigned int rsv[4];
};
};
#define ptp_perout_request compat_ptp_perout_request
#endif /*LINUX_VERSION_CODE < 5.8*/
#ifndef PTP_MAX_SAMPLES
#define PTP_MAX_SAMPLES 25 /* Maximum allowed offset measurement samples. */
#endif /* PTP_MAX_SAMPLES */

358
phc2sys.c
View File

@ -63,12 +63,6 @@
#define NS_PER_SEC 1000000000LL
#define PHC_PPS_OFFSET_LIMIT 10000000
#define PMC_UPDATE_INTERVAL (60 * NS_PER_SEC)
#define PMC_SUBSCRIBE_DURATION 180 /* 3 minutes */
/* Note that PMC_SUBSCRIBE_DURATION has to be longer than
* PMC_UPDATE_INTERVAL otherwise subscription will time out before it is
* renewed.
*/
struct clock {
LIST_ENTRY(clock) list;
@ -100,24 +94,6 @@ struct port {
struct clock *clock;
};
struct pmc_node;
typedef int pmc_node_recv_subscribed_t(struct pmc_node *node,
struct ptp_message *msg,
int excluded);
struct pmc_node {
struct pmc *pmc;
int pmc_ds_requested;
uint64_t pmc_last_update;
int sync_offset;
int leap;
int utc_offset_traceable;
int clock_identity_set;
struct ClockIdentity clock_identity;
pmc_node_recv_subscribed_t *recv_subscribed;
};
struct phc2sys_private {
unsigned int stats_max_count;
int sanity_freq_limit;
@ -136,17 +112,11 @@ struct phc2sys_private {
static struct config *phc2sys_config;
int update_pmc_node(struct pmc_node *node, int subscribe);
static int clock_handle_leap(struct phc2sys_private *priv,
struct clock *clock,
int64_t offset, uint64_t ts);
int run_pmc_get_utc_offset(struct pmc_node *node, int timeout);
void run_pmc_events(struct pmc_node *node);
static int normalize_state(int state);
int run_pmc_port_properties(struct pmc_node *node, int timeout,
unsigned int port, int *state,
int *tstamping, char *iface);
static struct servo *servo_add(struct phc2sys_private *priv,
struct clock *clock)
@ -803,56 +773,6 @@ static int do_loop(struct phc2sys_private *priv, int subscriptions)
return 0;
}
static int check_clock_identity(struct pmc_node *node, struct ptp_message *msg)
{
if (!node->clock_identity_set)
return 1;
return cid_eq(&node->clock_identity,
&msg->header.sourcePortIdentity.clockIdentity);
}
static int is_msg_mgt(struct ptp_message *msg)
{
struct TLV *tlv;
if (msg_type(msg) != MANAGEMENT)
return 0;
if (management_action(msg) != RESPONSE)
return 0;
if (msg_tlv_count(msg) != 1)
return 0;
tlv = (struct TLV *) msg->management.suffix;
if (tlv->type == TLV_MANAGEMENT)
return 1;
if (tlv->type == TLV_MANAGEMENT_ERROR_STATUS)
return -1;
return 0;
}
int get_mgt_id(struct ptp_message *msg)
{
struct management_tlv *mgt;
mgt = (struct management_tlv *) msg->management.suffix;
return mgt->id;
}
void *get_mgt_data(struct ptp_message *msg)
{
struct management_tlv *mgt;
mgt = (struct management_tlv *) msg->management.suffix;
return mgt->data;
}
static int get_mgt_err_id(struct ptp_message *msg)
{
struct management_error_status *mgt;
mgt = (struct management_error_status *)msg->management.suffix;
return mgt->id;
}
static int normalize_state(int state)
{
if (state != PS_MASTER && state != PS_SLAVE &&
@ -923,260 +843,6 @@ static int phc2sys_recv_subscribed(struct pmc_node *node,
return 0;
}
static void send_subscription(struct pmc_node *node)
{
struct subscribe_events_np sen;
memset(&sen, 0, sizeof(sen));
sen.duration = PMC_SUBSCRIBE_DURATION;
sen.bitmask[0] = 1 << NOTIFY_PORT_STATE;
pmc_send_set_action(node->pmc, TLV_SUBSCRIBE_EVENTS_NP, &sen, sizeof(sen));
}
int init_pmc_node(struct config *cfg, struct pmc_node *node, const char *uds,
pmc_node_recv_subscribed_t *recv_subscribed)
{
node->pmc = pmc_create(cfg, TRANS_UDS, uds, 0,
config_get_int(cfg, NULL, "domainNumber"),
config_get_int(cfg, NULL, "transportSpecific") << 4, 1);
if (!node->pmc) {
pr_err("failed to create pmc");
return -1;
}
node->recv_subscribed = recv_subscribed;
return 0;
}
/* Return values:
* 1: success
* 0: timeout
* -1: error reported by the other side
* -2: local error, fatal
*/
static int run_pmc(struct pmc_node *node, int timeout, int ds_id,
struct ptp_message **msg)
{
#define N_FD 1
struct pollfd pollfd[N_FD];
int cnt, res;
while (1) {
pollfd[0].fd = pmc_get_transport_fd(node->pmc);
pollfd[0].events = POLLIN|POLLPRI;
if (!node->pmc_ds_requested && ds_id >= 0)
pollfd[0].events |= POLLOUT;
cnt = poll(pollfd, N_FD, timeout);
if (cnt < 0) {
pr_err("poll failed");
return -2;
}
if (!cnt) {
/* Request the data set again in the next run. */
node->pmc_ds_requested = 0;
return 0;
}
/* Send a new request if there are no pending messages. */
if ((pollfd[0].revents & POLLOUT) &&
!(pollfd[0].revents & (POLLIN|POLLPRI))) {
switch (ds_id) {
case TLV_SUBSCRIBE_EVENTS_NP:
send_subscription(node);
break;
default:
pmc_send_get_action(node->pmc, ds_id);
break;
}
node->pmc_ds_requested = 1;
}
if (!(pollfd[0].revents & (POLLIN|POLLPRI)))
continue;
*msg = pmc_recv(node->pmc);
if (!*msg)
continue;
if (!check_clock_identity(node, *msg)) {
msg_put(*msg);
*msg = NULL;
continue;
}
res = is_msg_mgt(*msg);
if (res < 0 && get_mgt_err_id(*msg) == ds_id) {
node->pmc_ds_requested = 0;
return -1;
}
if (res <= 0 || node->recv_subscribed(node, *msg, ds_id) ||
get_mgt_id(*msg) != ds_id) {
msg_put(*msg);
*msg = NULL;
continue;
}
node->pmc_ds_requested = 0;
return 1;
}
}
static int run_pmc_wait_sync(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
Enumeration8 portState;
void *data;
int res;
while (1) {
res = run_pmc(node, timeout, TLV_PORT_DATA_SET, &msg);
if (res <= 0)
return res;
data = get_mgt_data(msg);
portState = ((struct portDS *)data)->portState;
msg_put(msg);
switch (portState) {
case PS_MASTER:
case PS_SLAVE:
return 1;
}
/* try to get more data sets (for other ports) */
node->pmc_ds_requested = 1;
}
}
int run_pmc_get_utc_offset(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
int res;
struct timePropertiesDS *tds;
res = run_pmc(node, timeout, TLV_TIME_PROPERTIES_DATA_SET, &msg);
if (res <= 0)
return res;
tds = (struct timePropertiesDS *)get_mgt_data(msg);
if (tds->flags & PTP_TIMESCALE) {
node->sync_offset = tds->currentUtcOffset;
if (tds->flags & LEAP_61)
node->leap = 1;
else if (tds->flags & LEAP_59)
node->leap = -1;
else
node->leap = 0;
node->utc_offset_traceable = tds->flags & UTC_OFF_VALID &&
tds->flags & TIME_TRACEABLE;
} else {
node->sync_offset = 0;
node->leap = 0;
node->utc_offset_traceable = 0;
}
msg_put(msg);
return 1;
}
int run_pmc_get_number_ports(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
int res;
struct defaultDS *dds;
res = run_pmc(node, timeout, TLV_DEFAULT_DATA_SET, &msg);
if (res <= 0)
return res;
dds = (struct defaultDS *)get_mgt_data(msg);
res = dds->numberPorts;
msg_put(msg);
return res;
}
int run_pmc_subscribe(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
int res;
res = run_pmc(node, timeout, TLV_SUBSCRIBE_EVENTS_NP, &msg);
if (res <= 0)
return res;
msg_put(msg);
return 1;
}
void run_pmc_events(struct pmc_node *node)
{
struct ptp_message *msg;
run_pmc(node, 0, -1, &msg);
}
int run_pmc_port_properties(struct pmc_node *node, int timeout,
unsigned int port, int *state,
int *tstamping, char *iface)
{
struct ptp_message *msg;
int res, len;
struct port_properties_np *ppn;
pmc_target_port(node->pmc, port);
while (1) {
res = run_pmc(node, timeout, TLV_PORT_PROPERTIES_NP, &msg);
if (res <= 0)
goto out;
ppn = get_mgt_data(msg);
if (ppn->portIdentity.portNumber != port) {
msg_put(msg);
continue;
}
*state = ppn->port_state;
*tstamping = ppn->timestamping;
len = ppn->interface.length;
if (len > IFNAMSIZ - 1)
len = IFNAMSIZ - 1;
memcpy(iface, ppn->interface.text, len);
iface[len] = '\0';
msg_put(msg);
res = 1;
break;
}
out:
pmc_target_all(node->pmc);
return res;
}
static int run_pmc_clock_identity(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
struct defaultDS *dds;
int res;
res = run_pmc(node, timeout, TLV_DEFAULT_DATA_SET, &msg);
if (res <= 0)
return res;
dds = (struct defaultDS *)get_mgt_data(msg);
memcpy(&node->clock_identity, &dds->clockIdentity,
sizeof(struct ClockIdentity));
node->clock_identity_set = 1;
msg_put(msg);
return 1;
}
void close_pmc_node(struct pmc_node *node)
{
if (!node->pmc)
return;
pmc_destroy(node->pmc);
node->pmc = NULL;
}
static int auto_init_ports(struct phc2sys_private *priv, int add_rt)
{
struct port *port;
@ -1255,30 +921,6 @@ static int auto_init_ports(struct phc2sys_private *priv, int add_rt)
return 0;
}
/* Returns: -1 in case of error, 0 otherwise */
int update_pmc_node(struct pmc_node *node, int subscribe)
{
struct timespec tp;
uint64_t ts;
if (clock_gettime(CLOCK_MONOTONIC, &tp)) {
pr_err("failed to read clock: %m");
return -1;
}
ts = tp.tv_sec * NS_PER_SEC + tp.tv_nsec;
if (node->pmc &&
!(ts > node->pmc_last_update &&
ts - node->pmc_last_update < PMC_UPDATE_INTERVAL)) {
if (subscribe)
run_pmc_subscribe(node, 0);
if (run_pmc_get_utc_offset(node, 0) > 0)
node->pmc_last_update = ts;
}
return 0;
}
/* Returns: non-zero to skip clock update */
static int clock_handle_leap(struct phc2sys_private *priv, struct clock *clock,
int64_t offset, uint64_t ts)

View File

@ -18,6 +18,8 @@
* 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
*/
#include <errno.h>
#include <net/if.h>
#include <poll.h>
#include <stdlib.h>
#include <string.h>
#include <sys/types.h>
@ -56,6 +58,13 @@
/* Includes one extra byte to make length even. */
#define EMPTY_PTP_TEXT 2
#define PMC_UPDATE_INTERVAL (60 * NS_PER_SEC)
#define PMC_SUBSCRIBE_DURATION 180 /* 3 minutes */
/* Note that PMC_SUBSCRIBE_DURATION has to be longer than
* PMC_UPDATE_INTERVAL otherwise subscription will time out before it is
* renewed.
*/
static void do_get_action(struct pmc *pmc, int action, int index, char *str);
static void do_set_action(struct pmc *pmc, int action, int index, char *str);
static void not_supported(struct pmc *pmc, int action, int index, char *str);
@ -711,3 +720,331 @@ int pmc_do_command(struct pmc *pmc, char *str)
return 0;
}
static void send_subscription(struct pmc_node *node)
{
struct subscribe_events_np sen;
memset(&sen, 0, sizeof(sen));
sen.duration = PMC_SUBSCRIBE_DURATION;
sen.bitmask[0] = 1 << NOTIFY_PORT_STATE;
pmc_send_set_action(node->pmc, TLV_SUBSCRIBE_EVENTS_NP, &sen, sizeof(sen));
}
static int check_clock_identity(struct pmc_node *node, struct ptp_message *msg)
{
if (!node->clock_identity_set)
return 1;
return cid_eq(&node->clock_identity,
&msg->header.sourcePortIdentity.clockIdentity);
}
static int is_msg_mgt(struct ptp_message *msg)
{
struct TLV *tlv;
if (msg_type(msg) != MANAGEMENT)
return 0;
if (management_action(msg) != RESPONSE)
return 0;
if (msg_tlv_count(msg) != 1)
return 0;
tlv = (struct TLV *) msg->management.suffix;
if (tlv->type == TLV_MANAGEMENT)
return 1;
if (tlv->type == TLV_MANAGEMENT_ERROR_STATUS)
return -1;
return 0;
}
int get_mgt_id(struct ptp_message *msg)
{
struct management_tlv *mgt;
mgt = (struct management_tlv *) msg->management.suffix;
return mgt->id;
}
void *get_mgt_data(struct ptp_message *msg)
{
struct management_tlv *mgt;
mgt = (struct management_tlv *) msg->management.suffix;
return mgt->data;
}
static int get_mgt_err_id(struct ptp_message *msg)
{
struct management_error_status *mgt;
mgt = (struct management_error_status *)msg->management.suffix;
return mgt->id;
}
/* Return values:
* 1: success
* 0: timeout
* -1: error reported by the other side
* -2: local error, fatal
*/
static int run_pmc(struct pmc_node *node, int timeout, int ds_id,
struct ptp_message **msg)
{
#define N_FD 1
struct pollfd pollfd[N_FD];
int cnt, res;
while (1) {
pollfd[0].fd = pmc_get_transport_fd(node->pmc);
pollfd[0].events = POLLIN|POLLPRI;
if (!node->pmc_ds_requested && ds_id >= 0)
pollfd[0].events |= POLLOUT;
cnt = poll(pollfd, N_FD, timeout);
if (cnt < 0) {
pr_err("poll failed");
return -2;
}
if (!cnt) {
/* Request the data set again in the next run. */
node->pmc_ds_requested = 0;
return 0;
}
/* Send a new request if there are no pending messages. */
if ((pollfd[0].revents & POLLOUT) &&
!(pollfd[0].revents & (POLLIN|POLLPRI))) {
switch (ds_id) {
case TLV_SUBSCRIBE_EVENTS_NP:
send_subscription(node);
break;
default:
pmc_send_get_action(node->pmc, ds_id);
break;
}
node->pmc_ds_requested = 1;
}
if (!(pollfd[0].revents & (POLLIN|POLLPRI)))
continue;
*msg = pmc_recv(node->pmc);
if (!*msg)
continue;
if (!check_clock_identity(node, *msg)) {
msg_put(*msg);
*msg = NULL;
continue;
}
res = is_msg_mgt(*msg);
if (res < 0 && get_mgt_err_id(*msg) == ds_id) {
node->pmc_ds_requested = 0;
return -1;
}
if (res <= 0 || node->recv_subscribed(node, *msg, ds_id) ||
get_mgt_id(*msg) != ds_id) {
msg_put(*msg);
*msg = NULL;
continue;
}
node->pmc_ds_requested = 0;
return 1;
}
}
int run_pmc_wait_sync(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
Enumeration8 portState;
void *data;
int res;
while (1) {
res = run_pmc(node, timeout, TLV_PORT_DATA_SET, &msg);
if (res <= 0)
return res;
data = get_mgt_data(msg);
portState = ((struct portDS *)data)->portState;
msg_put(msg);
switch (portState) {
case PS_MASTER:
case PS_SLAVE:
return 1;
}
/* try to get more data sets (for other ports) */
node->pmc_ds_requested = 1;
}
}
int run_pmc_get_utc_offset(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
int res;
struct timePropertiesDS *tds;
res = run_pmc(node, timeout, TLV_TIME_PROPERTIES_DATA_SET, &msg);
if (res <= 0)
return res;
tds = (struct timePropertiesDS *)get_mgt_data(msg);
if (tds->flags & PTP_TIMESCALE) {
node->sync_offset = tds->currentUtcOffset;
if (tds->flags & LEAP_61)
node->leap = 1;
else if (tds->flags & LEAP_59)
node->leap = -1;
else
node->leap = 0;
node->utc_offset_traceable = tds->flags & UTC_OFF_VALID &&
tds->flags & TIME_TRACEABLE;
} else {
node->sync_offset = 0;
node->leap = 0;
node->utc_offset_traceable = 0;
}
msg_put(msg);
return 1;
}
int run_pmc_get_number_ports(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
int res;
struct defaultDS *dds;
res = run_pmc(node, timeout, TLV_DEFAULT_DATA_SET, &msg);
if (res <= 0)
return res;
dds = (struct defaultDS *)get_mgt_data(msg);
res = dds->numberPorts;
msg_put(msg);
return res;
}
int run_pmc_subscribe(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
int res;
res = run_pmc(node, timeout, TLV_SUBSCRIBE_EVENTS_NP, &msg);
if (res <= 0)
return res;
msg_put(msg);
return 1;
}
void run_pmc_events(struct pmc_node *node)
{
struct ptp_message *msg;
run_pmc(node, 0, -1, &msg);
}
int run_pmc_port_properties(struct pmc_node *node, int timeout,
unsigned int port, int *state,
int *tstamping, char *iface)
{
struct ptp_message *msg;
int res, len;
struct port_properties_np *ppn;
pmc_target_port(node->pmc, port);
while (1) {
res = run_pmc(node, timeout, TLV_PORT_PROPERTIES_NP, &msg);
if (res <= 0)
goto out;
ppn = get_mgt_data(msg);
if (ppn->portIdentity.portNumber != port) {
msg_put(msg);
continue;
}
*state = ppn->port_state;
*tstamping = ppn->timestamping;
len = ppn->interface.length;
if (len > IFNAMSIZ - 1)
len = IFNAMSIZ - 1;
memcpy(iface, ppn->interface.text, len);
iface[len] = '\0';
msg_put(msg);
res = 1;
break;
}
out:
pmc_target_all(node->pmc);
return res;
}
int run_pmc_clock_identity(struct pmc_node *node, int timeout)
{
struct ptp_message *msg;
struct defaultDS *dds;
int res;
res = run_pmc(node, timeout, TLV_DEFAULT_DATA_SET, &msg);
if (res <= 0)
return res;
dds = (struct defaultDS *)get_mgt_data(msg);
memcpy(&node->clock_identity, &dds->clockIdentity,
sizeof(struct ClockIdentity));
node->clock_identity_set = 1;
msg_put(msg);
return 1;
}
/* Returns: -1 in case of error, 0 otherwise */
int update_pmc_node(struct pmc_node *node, int subscribe)
{
struct timespec tp;
uint64_t ts;
if (clock_gettime(CLOCK_MONOTONIC, &tp)) {
pr_err("failed to read clock: %m");
return -1;
}
ts = tp.tv_sec * NS_PER_SEC + tp.tv_nsec;
if (node->pmc &&
!(ts > node->pmc_last_update &&
ts - node->pmc_last_update < PMC_UPDATE_INTERVAL)) {
if (subscribe)
run_pmc_subscribe(node, 0);
if (run_pmc_get_utc_offset(node, 0) > 0)
node->pmc_last_update = ts;
}
return 0;
}
int init_pmc_node(struct config *cfg, struct pmc_node *node, const char *uds,
pmc_node_recv_subscribed_t *recv_subscribed)
{
node->pmc = pmc_create(cfg, TRANS_UDS, uds, 0,
config_get_int(cfg, NULL, "domainNumber"),
config_get_int(cfg, NULL, "transportSpecific") << 4, 1);
if (!node->pmc) {
pr_err("failed to create pmc");
return -1;
}
node->recv_subscribed = recv_subscribed;
return 0;
}
void close_pmc_node(struct pmc_node *node)
{
if (!node->pmc)
return;
pmc_destroy(node->pmc);
node->pmc = NULL;
}

View File

@ -22,6 +22,7 @@
#define HAVE_PMC_COMMON_H
#include "config.h"
#include "fsm.h"
#include "msg.h"
#include "transport.h"
@ -49,4 +50,38 @@ void pmc_target_all(struct pmc *pmc);
const char *pmc_action_string(int action);
int pmc_do_command(struct pmc *pmc, char *str);
struct pmc_node;
typedef int pmc_node_recv_subscribed_t(struct pmc_node *node,
struct ptp_message *msg,
int excluded);
struct pmc_node {
struct pmc *pmc;
int pmc_ds_requested;
uint64_t pmc_last_update;
int sync_offset;
int leap;
int utc_offset_traceable;
int clock_identity_set;
struct ClockIdentity clock_identity;
pmc_node_recv_subscribed_t *recv_subscribed;
};
int init_pmc_node(struct config *cfg, struct pmc_node *node, const char *uds,
pmc_node_recv_subscribed_t *recv_subscribed);
void close_pmc_node(struct pmc_node *node);
int update_pmc_node(struct pmc_node *node, int subscribe);
int run_pmc_subscribe(struct pmc_node *node, int timeout);
int run_pmc_clock_identity(struct pmc_node *node, int timeout);
int run_pmc_wait_sync(struct pmc_node *node, int timeout);
int run_pmc_get_number_ports(struct pmc_node *node, int timeout);
void run_pmc_events(struct pmc_node *node);
int run_pmc_port_properties(struct pmc_node *node, int timeout,
unsigned int port, int *state,
int *tstamping, char *iface);
int run_pmc_get_utc_offset(struct pmc_node *node, int timeout);
int get_mgt_id(struct ptp_message *msg);
void *get_mgt_data(struct ptp_message *msg);
#endif

View File

@ -26,6 +26,21 @@ A single source may be used to distribute time to one or more PHC devices.
.SH OPTIONS
.TP
.BI \-a
Adjust the direction of synchronization automatically. The program determines
which PHC should be a source of time and which should be a sink by querying the
port states from the running instance of
.B ptp4l.
Note that using this option, the PPS signal distribution hierarchy still
remains fixed as per the configuration file. This implies that using this
option, a PHC PPS master may become a time sink, and a PPS slave may become a
time source. Other, non-PHC types of PPS masters (generic, NMEA) cannot become
time sinks. Clocks which are not part of
.B ptp4l's
list of ports are not synchronized. This option is useful when the
.B boundary_clock_jbod
option of ptp4l is also enabled.
.TP
.BI \-c " device|name"
Specifies a PHC slave clock to be synchronized.
The clock may be identified by its character device (like /dev/ptp0)
@ -152,6 +167,11 @@ specified, then the given remote connection will be used in preference
to the configured serial port.
The default is "/dev/ttyS0".
.TP
.B ts2phc.perout_phase
Configures the offset between the beginning of the second and the PPS
master's rising edge. Available only for a PHC master. The supported
range is 0 to 999999999 nanoseconds. The default is 0 nanoseconds.
.TP
.B ts2phc.pulsewidth
The expected pulse width of the external PPS signal in nanoseconds.
When 'ts2phc.extts_polarity' is "both", the given pulse width is used

638
ts2phc.c
View File

@ -7,27 +7,524 @@
* @note SPDX-License-Identifier: GPL-2.0+
*/
#include <stdlib.h>
#include <net/if.h>
#include <sys/types.h>
#include <unistd.h>
#include "clockadj.h"
#include "config.h"
#include "contain.h"
#include "interface.h"
#include "phc.h"
#include "print.h"
#include "ts2phc_master.h"
#include "ts2phc_slave.h"
#include "ts2phc.h"
#include "version.h"
#define NS_PER_SEC 1000000000LL
#define SAMPLE_WEIGHT 1.0
struct interface {
STAILQ_ENTRY(interface) list;
};
static void ts2phc_cleanup(struct config *cfg, struct ts2phc_master *master)
static void ts2phc_cleanup(struct ts2phc_private *priv)
{
ts2phc_slave_cleanup();
if (master) {
ts2phc_master_destroy(master);
struct port *p, *tmp;
ts2phc_slave_cleanup(priv);
if (priv->master)
ts2phc_master_destroy(priv->master);
if (priv->cfg)
config_destroy(priv->cfg);
close_pmc_node(&priv->node);
/*
* Clocks are destroyed by the cleanup methods of the individual
* master and slave PHC modules.
*/
LIST_FOREACH_SAFE(p, &priv->ports, list, tmp)
free(p);
msg_cleanup();
}
/* FIXME: Copied from phc2sys */
static int normalize_state(int state)
{
if (state != PS_MASTER && state != PS_SLAVE &&
state != PS_PRE_MASTER && state != PS_UNCALIBRATED) {
/* treat any other state as "not a master nor a slave" */
state = PS_DISABLED;
}
if (cfg) {
config_destroy(cfg);
return state;
}
/* FIXME: Copied from phc2sys */
static struct port *port_get(struct ts2phc_private *priv, unsigned int number)
{
struct port *p;
LIST_FOREACH(p, &priv->ports, list) {
if (p->number == number)
return p;
}
return NULL;
}
/* FIXME: Copied from phc2sys */
static int clock_compute_state(struct ts2phc_private *priv,
struct clock *clock)
{
int state = PS_DISABLED;
struct port *p;
LIST_FOREACH(p, &priv->ports, list) {
if (p->clock != clock)
continue;
/* PS_SLAVE takes the highest precedence, PS_UNCALIBRATED
* after that, PS_MASTER is third, PS_PRE_MASTER fourth and
* all of that overrides PS_DISABLED, which corresponds
* nicely with the numerical values */
if (p->state > state)
state = p->state;
}
return state;
}
#define node_to_ts2phc(node) \
container_of(node, struct ts2phc_private, node)
static int ts2phc_recv_subscribed(struct pmc_node *node,
struct ptp_message *msg, int excluded)
{
struct ts2phc_private *priv = node_to_ts2phc(node);
int mgt_id, state;
struct portDS *pds;
struct port *port;
struct clock *clock;
mgt_id = get_mgt_id(msg);
if (mgt_id == excluded)
return 0;
switch (mgt_id) {
case TLV_PORT_DATA_SET:
pds = get_mgt_data(msg);
port = port_get(priv, pds->portIdentity.portNumber);
if (!port) {
pr_info("received data for unknown port %s",
pid2str(&pds->portIdentity));
return 1;
}
state = normalize_state(pds->portState);
if (port->state != state) {
pr_info("port %s changed state",
pid2str(&pds->portIdentity));
port->state = state;
clock = port->clock;
state = clock_compute_state(priv, clock);
if (clock->state != state || clock->new_state) {
clock->new_state = state;
priv->state_changed = 1;
}
}
return 1;
}
return 0;
}
struct servo *servo_add(struct ts2phc_private *priv, struct clock *clock)
{
enum servo_type type = config_get_int(priv->cfg, NULL, "clock_servo");
struct servo *servo;
int fadj, max_adj;
fadj = (int) clockadj_get_freq(clock->clkid);
/* Due to a bug in older kernels, the reading may silently fail
and return 0. Set the frequency back to make sure fadj is
the actual frequency of the clock. */
clockadj_set_freq(clock->clkid, fadj);
max_adj = phc_max_adj(clock->clkid);
servo = servo_create(priv->cfg, type, -fadj, max_adj, 0);
if (!servo)
return NULL;
servo_sync_interval(servo, SERVO_SYNC_INTERVAL);
return servo;
}
void clock_add_tstamp(struct clock *clock, tmv_t t)
{
struct timespec ts = tmv_to_timespec(t);
pr_debug("adding tstamp %ld.%09ld to clock %s",
ts.tv_sec, ts.tv_nsec, clock->name);
clock->last_ts = t;
clock->is_ts_available = 1;
}
static int clock_get_tstamp(struct clock *clock, tmv_t *ts)
{
if (!clock->is_ts_available)
return 0;
clock->is_ts_available = 0;
*ts = clock->last_ts;
return 1;
}
static void clock_flush_tstamp(struct clock *clock)
{
clock->is_ts_available = 0;
}
struct clock *clock_add(struct ts2phc_private *priv, const char *device)
{
clockid_t clkid = CLOCK_INVALID;
int phc_index = -1;
struct clock *c;
int err;
clkid = posix_clock_open(device, &phc_index);
if (clkid == CLOCK_INVALID)
return NULL;
LIST_FOREACH(c, &priv->clocks, list) {
if (c->phc_index == phc_index) {
/* Already have the clock, don't add it again */
posix_clock_close(clkid);
return c;
}
}
c = calloc(1, sizeof(*c));
if (!c) {
pr_err("failed to allocate memory for a clock");
return NULL;
}
c->clkid = clkid;
c->phc_index = phc_index;
c->servo_state = SERVO_UNLOCKED;
c->servo = servo_add(priv, c);
c->no_adj = config_get_int(priv->cfg, NULL, "free_running");
err = asprintf(&c->name, "/dev/ptp%d", phc_index);
if (err < 0) {
free(c);
posix_clock_close(clkid);
return NULL;
}
LIST_INSERT_HEAD(&priv->clocks, c, list);
return c;
}
void clock_destroy(struct clock *c)
{
servo_destroy(c->servo);
posix_clock_close(c->clkid);
free(c->name);
free(c);
}
/* FIXME: Copied from phc2sys */
static struct port *port_add(struct ts2phc_private *priv, unsigned int number,
char *device)
{
struct clock *c = NULL;
struct port *p, *tmp;
p = port_get(priv, number);
if (p)
return p;
/* port is a new one, look whether we have the device already on
* a different port */
LIST_FOREACH(tmp, &priv->ports, list) {
if (tmp->number == number) {
c = tmp->clock;
break;
}
}
if (!c) {
c = clock_add(priv, device);
if (!c)
return NULL;
}
p = malloc(sizeof(*p));
if (!p) {
pr_err("failed to allocate memory for a port");
clock_destroy(c);
return NULL;
}
p->number = number;
p->clock = c;
LIST_INSERT_HEAD(&priv->ports, p, list);
return p;
}
static int auto_init_ports(struct ts2phc_private *priv)
{
int state, timestamping;
int number_ports, res;
char iface[IFNAMSIZ];
struct clock *clock;
struct port *port;
unsigned int i;
while (1) {
if (!is_running())
return -1;
res = run_pmc_clock_identity(&priv->node, 1000);
if (res < 0)
return -1;
if (res > 0)
break;
/* res == 0, timeout */
pr_notice("Waiting for ptp4l...");
}
number_ports = run_pmc_get_number_ports(&priv->node, 1000);
if (number_ports <= 0) {
pr_err("failed to get number of ports");
return -1;
}
res = run_pmc_subscribe(&priv->node, 1000);
if (res <= 0) {
pr_err("failed to subscribe");
return -1;
}
for (i = 1; i <= number_ports; i++) {
res = run_pmc_port_properties(&priv->node, 1000, i, &state,
&timestamping, iface);
if (res == -1) {
/* port does not exist, ignore the port */
continue;
}
if (res <= 0) {
pr_err("failed to get port properties");
return -1;
}
if (timestamping == TS_SOFTWARE) {
/* ignore ports with software time stamping */
continue;
}
port = port_add(priv, i, iface);
if (!port)
return -1;
port->state = normalize_state(state);
}
if (LIST_EMPTY(&priv->clocks)) {
pr_err("no suitable ports available");
return -1;
}
LIST_FOREACH(clock, &priv->clocks, list) {
clock->new_state = clock_compute_state(priv, clock);
}
priv->state_changed = 1;
return 0;
}
static void ts2phc_reconfigure(struct ts2phc_private *priv)
{
struct clock *c, *src = NULL, *last = NULL;
int src_cnt = 0, dst_cnt = 0;
pr_info("reconfiguring after port state change");
priv->state_changed = 0;
LIST_FOREACH(c, &priv->clocks, list) {
if (c->new_state) {
c->state = c->new_state;
c->new_state = 0;
}
switch (c->state) {
case PS_FAULTY:
case PS_DISABLED:
case PS_LISTENING:
case PS_PRE_MASTER:
case PS_MASTER:
case PS_PASSIVE:
if (!c->is_destination) {
pr_info("selecting %s for synchronization",
c->name);
c->is_destination = 1;
}
dst_cnt++;
break;
case PS_UNCALIBRATED:
src_cnt++;
break;
case PS_SLAVE:
src = c;
src_cnt++;
break;
}
last = c;
}
if (dst_cnt >= 1 && !src) {
priv->source = last;
priv->source->is_destination = 0;
/* Reset to original state in next reconfiguration. */
priv->source->new_state = priv->source->state;
priv->source->state = PS_SLAVE;
pr_info("no source, selecting %s as the default clock",
last->name);
return;
}
if (src_cnt > 1) {
pr_info("multiple source clocks available, postponing sync...");
priv->source = NULL;
return;
}
if (src_cnt > 0 && !src) {
pr_info("source clock not ready, waiting...");
priv->source = NULL;
return;
}
if (!src_cnt && !dst_cnt) {
pr_info("no PHC ready, waiting...");
priv->source = NULL;
return;
}
if (!src_cnt) {
pr_info("nothing to synchronize");
priv->source = NULL;
return;
}
src->is_destination = 0;
priv->source = src;
pr_info("selecting %s as the source clock", src->name);
}
static int ts2phc_approximate_master_tstamp(struct ts2phc_private *priv,
tmv_t *master_tmv)
{
struct timespec master_ts;
tmv_t tmv;
int err;
err = ts2phc_master_getppstime(priv->master, &master_ts);
if (err < 0) {
pr_err("master ts not valid");
return err;
}
tmv = timespec_to_tmv(master_ts);
tmv = tmv_sub(tmv, priv->perout_phase);
master_ts = tmv_to_timespec(tmv);
/*
* As long as the kernel doesn't support a proper API for reporting
* a precise perout timestamp, we'll have to use this crude
* approximation.
*/
if (master_ts.tv_nsec > NS_PER_SEC / 2)
master_ts.tv_sec++;
master_ts.tv_nsec = 0;
tmv = timespec_to_tmv(master_ts);
tmv = tmv_add(tmv, priv->perout_phase);
*master_tmv = tmv;
return 0;
}
static void ts2phc_synchronize_clocks(struct ts2phc_private *priv, int autocfg)
{
tmv_t source_tmv;
struct clock *c;
int valid, err;
if (autocfg) {
if (!priv->source) {
pr_debug("no source, skipping");
return;
}
valid = clock_get_tstamp(priv->source, &source_tmv);
if (!valid) {
pr_err("source clock (%s) timestamp not valid, skipping",
priv->source->name);
return;
}
} else {
err = ts2phc_approximate_master_tstamp(priv, &source_tmv);
if (err < 0)
return;
}
LIST_FOREACH(c, &priv->clocks, list) {
int64_t offset;
double adj;
tmv_t ts;
if (!c->is_destination)
continue;
valid = clock_get_tstamp(c, &ts);
if (!valid) {
pr_debug("%s timestamp not valid, skipping", c->name);
continue;
}
offset = tmv_to_nanoseconds(tmv_sub(ts, source_tmv));
if (c->no_adj) {
pr_info("%s offset %10" PRId64, c->name,
offset);
continue;
}
adj = servo_sample(c->servo, offset, tmv_to_nanoseconds(ts),
SAMPLE_WEIGHT, &c->servo_state);
pr_info("%s offset %10" PRId64 " s%d freq %+7.0f",
c->name, offset, c->servo_state, adj);
switch (c->servo_state) {
case SERVO_UNLOCKED:
break;
case SERVO_JUMP:
clockadj_set_freq(c->clkid, -adj);
clockadj_step(c->clkid, -offset);
break;
case SERVO_LOCKED:
case SERVO_LOCKED_STABLE:
clockadj_set_freq(c->clkid, -adj);
break;
}
}
}
static int ts2phc_collect_master_tstamp(struct ts2phc_private *priv)
{
struct clock *master_clock;
tmv_t master_tmv;
int err;
master_clock = ts2phc_master_get_clock(priv->master);
/*
* Master isn't a PHC (it may be a generic or a GPS master),
* don't error out, just don't do anything. If it doesn't have a PHC,
* there is nothing to synchronize, which is the only point of
* collecting its perout timestamp in the first place.
*/
if (!master_clock)
return 0;
err = ts2phc_approximate_master_tstamp(priv, &master_tmv);
if (err < 0)
return err;
clock_add_tstamp(master_clock, master_tmv);
return 0;
}
static void usage(char *progname)
@ -35,6 +532,7 @@ static void usage(char *progname)
fprintf(stderr,
"\n"
"usage: %s [options]\n\n"
" -a turn on autoconfiguration\n"
" -c [dev|name] phc slave clock (like /dev/ptp0 or eth0)\n"
" (may be specified multiple times)\n"
" -f [file] read configuration from 'file'\n"
@ -56,19 +554,21 @@ static void usage(char *progname)
int main(int argc, char *argv[])
{
int c, err = 0, have_slave = 0, index, print_level;
struct ts2phc_master *master = NULL;
char uds_local[MAX_IFNAME_SIZE + 1];
enum ts2phc_master_type pps_type;
struct ts2phc_private priv = {0};
char *config = NULL, *progname;
const char *pps_source = NULL;
struct config *cfg = NULL;
struct interface *iface;
struct option *opts;
int autocfg = 0;
handle_term_signals();
cfg = config_create();
if (!cfg) {
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
@ -77,18 +577,21 @@ int main(int argc, char *argv[])
/* Process the command line arguments. */
progname = strrchr(argv[0], '/');
progname = progname ? 1 + progname : argv[0];
while (EOF != (c = getopt_long(argc, argv, "c:f:hi:l:mqs:v", opts, &index))) {
while (EOF != (c = getopt_long(argc, argv, "ac:f:hi:l:mqs:v", opts, &index))) {
switch (c) {
case 0:
if (config_parse_option(cfg, opts[index].name, optarg)) {
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
break;
case 'a':
autocfg = 1;
break;
case 'c':
if (!config_create_interface(optarg, cfg)) {
fprintf(stderr, "failed to add slave\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
have_slave = 1;
@ -99,7 +602,7 @@ int main(int argc, char *argv[])
case 'l':
if (get_arg_val_i(c, optarg, &print_level,
PRINT_LEVEL_MIN, PRINT_LEVEL_MAX)) {
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
config_set_int(cfg, "logging_level", print_level);
@ -116,29 +619,29 @@ int main(int argc, char *argv[])
case 's':
if (pps_source) {
fprintf(stderr, "too many PPS sources\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
pps_source = optarg;
break;
case 'v':
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
version_show(stdout);
return 0;
case 'h':
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
usage(progname);
return -1;
case '?':
default:
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
usage(progname);
return -1;
}
}
if (config && (c = config_read(config, cfg))) {
fprintf(stderr, "failed to read config\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
print_set_progname(progname);
@ -147,18 +650,54 @@ int main(int argc, char *argv[])
print_set_syslog(config_get_int(cfg, NULL, "use_syslog"));
print_set_level(config_get_int(cfg, NULL, "logging_level"));
STAILQ_INIT(&priv.slaves);
priv.cfg = cfg;
snprintf(uds_local, sizeof(uds_local), "/var/run/ts2phc.%d",
getpid());
if (autocfg) {
err = init_pmc_node(cfg, &priv.node, uds_local,
ts2phc_recv_subscribed);
if (err) {
ts2phc_cleanup(&priv);
return -1;
}
err = auto_init_ports(&priv);
if (err) {
ts2phc_cleanup(&priv);
return -1;
}
}
STAILQ_FOREACH(iface, &cfg->interfaces, list) {
if (1 == config_get_int(cfg, interface_name(iface), "ts2phc.master")) {
const char *dev = interface_name(iface);
if (1 == config_get_int(cfg, dev, "ts2phc.master")) {
int perout_phase;
if (pps_source) {
fprintf(stderr, "too many PPS sources\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
pps_source = interface_name(iface);
pps_source = dev;
perout_phase = config_get_int(cfg, dev,
"ts2phc.perout_phase");
/*
* We use a default value of -1 to distinguish whether
* to use the PTP_PEROUT_PHASE API or not. But if we
* don't use that (and therefore we use absolute start
* time), the phase is still zero, by our application's
* convention.
*/
if (perout_phase < 0)
perout_phase = 0;
priv.perout_phase = nanoseconds_to_tmv(perout_phase);
} else {
if (ts2phc_slave_add(cfg, interface_name(iface))) {
if (ts2phc_slave_add(&priv, interface_name(iface))) {
fprintf(stderr, "failed to add slave\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
have_slave = 1;
@ -166,19 +705,19 @@ int main(int argc, char *argv[])
}
if (!have_slave) {
fprintf(stderr, "no slave clocks specified\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
usage(progname);
return -1;
}
if (!pps_source) {
fprintf(stderr, "no PPS source specified\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
usage(progname);
return -1;
}
if (ts2phc_slave_arm()) {
fprintf(stderr, "failed to arm slaves\n");
ts2phc_cleanup(cfg, master);
if (ts2phc_slaves_init(&priv)) {
fprintf(stderr, "failed to initialize slaves\n");
ts2phc_cleanup(&priv);
return -1;
}
@ -189,21 +728,50 @@ int main(int argc, char *argv[])
} else {
pps_type = TS2PHC_MASTER_PHC;
}
master = ts2phc_master_create(cfg, pps_source, pps_type);
if (!master) {
priv.master = ts2phc_master_create(&priv, pps_source, pps_type);
if (!priv.master) {
fprintf(stderr, "failed to create master\n");
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return -1;
}
while (is_running()) {
err = ts2phc_slave_poll(master);
if (err) {
struct clock *c;
if (autocfg) {
/*
* Make sure ptp4l sees us as alive and doesn't prune
* us from the list of subscribers
*/
err = update_pmc_node(&priv.node, 1);
if (err < 0) {
pr_err("update_pmc_node returned %d", err);
break;
}
run_pmc_events(&priv.node);
if (priv.state_changed)
ts2phc_reconfigure(&priv);
}
LIST_FOREACH(c, &priv.clocks, list)
clock_flush_tstamp(c);
err = ts2phc_slave_poll(&priv);
if (err < 0) {
pr_err("poll failed");
break;
}
if (err > 0) {
err = ts2phc_collect_master_tstamp(&priv);
if (err) {
pr_err("failed to collect master tstamp");
break;
}
ts2phc_synchronize_clocks(&priv, autocfg);
}
}
ts2phc_cleanup(cfg, master);
ts2phc_cleanup(&priv);
return err;
}

64
ts2phc.h 100644
View File

@ -0,0 +1,64 @@
/**
* @file ts2phc.h
* @brief Structure definitions for ts2phc
* @note Copyright 2020 Vladimir Oltean <olteanv@gmail.com>
* @note SPDX-License-Identifier: GPL-2.0+
*/
#ifndef HAVE_TS2PHC_H
#define HAVE_TS2PHC_H
#include <sys/queue.h>
#include <time.h>
#include "pmc_common.h"
#include "servo.h"
struct ts2phc_slave_array;
#define SERVO_SYNC_INTERVAL 1.0
struct clock {
LIST_ENTRY(clock) list;
LIST_ENTRY(clock) dst_list;
clockid_t clkid;
int phc_index;
int state;
int new_state;
struct servo *servo;
enum servo_state servo_state;
char *name;
int no_adj;
int is_destination;
int is_ts_available;
tmv_t last_ts;
};
struct port {
LIST_ENTRY(port) list;
unsigned int number;
int state;
struct clock *clock;
};
struct ts2phc_private {
struct ts2phc_master *master;
STAILQ_HEAD(slave_ifaces_head, ts2phc_slave) slaves;
unsigned int n_slaves;
struct ts2phc_slave_array *polling_array;
tmv_t perout_phase;
struct config *cfg;
struct pmc_node node;
int state_changed;
struct clock *source;
LIST_HEAD(port_head, port) ports;
LIST_HEAD(clock_head, clock) clocks;
};
struct servo *servo_add(struct ts2phc_private *priv, struct clock *clock);
struct clock *clock_add(struct ts2phc_private *priv, const char *device);
void clock_add_tstamp(struct clock *clock, tmv_t ts);
void clock_destroy(struct clock *clock);
#include "ts2phc_master.h"
#include "ts2phc_slave.h"
#endif

View File

@ -47,7 +47,7 @@ static int ts2phc_generic_master_getppstime(struct ts2phc_master *m,
return 0;
}
struct ts2phc_master *ts2phc_generic_master_create(struct config *cfg,
struct ts2phc_master *ts2phc_generic_master_create(struct ts2phc_private *priv,
const char *dev)
{
struct ts2phc_generic_master *master;

View File

@ -6,9 +6,10 @@
#ifndef HAVE_TS2PHC_GENERIC_MASTER_H
#define HAVE_TS2PHC_GENERIC_MASTER_H
#include "ts2phc.h"
#include "ts2phc_master.h"
struct ts2phc_master *ts2phc_generic_master_create(struct config *cfg,
struct ts2phc_master *ts2phc_generic_master_create(struct ts2phc_private *priv,
const char *dev);
#endif

View File

@ -3,25 +3,27 @@
* @note Copyright (C) 2019 Richard Cochran <richardcochran@gmail.com>
* @note SPDX-License-Identifier: GPL-2.0+
*/
#include "ts2phc.h"
#include "ts2phc_generic_master.h"
#include "ts2phc_master_private.h"
#include "ts2phc_nmea_master.h"
#include "ts2phc_phc_master.h"
struct ts2phc_master *ts2phc_master_create(struct config *cfg, const char *dev,
struct ts2phc_master *ts2phc_master_create(struct ts2phc_private *priv,
const char *dev,
enum ts2phc_master_type type)
{
struct ts2phc_master *master = NULL;
switch (type) {
case TS2PHC_MASTER_GENERIC:
master = ts2phc_generic_master_create(cfg, dev);
master = ts2phc_generic_master_create(priv, dev);
break;
case TS2PHC_MASTER_NMEA:
master = ts2phc_nmea_master_create(cfg, dev);
master = ts2phc_nmea_master_create(priv, dev);
break;
case TS2PHC_MASTER_PHC:
master = ts2phc_phc_master_create(cfg, dev);
master = ts2phc_phc_master_create(priv, dev);
break;
}
return master;
@ -36,3 +38,11 @@ int ts2phc_master_getppstime(struct ts2phc_master *master, struct timespec *ts)
{
return master->getppstime(master, ts);
}
struct clock *ts2phc_master_get_clock(struct ts2phc_master *m)
{
if (m->get_clock)
return m->get_clock(m);
return NULL;
}

View File

@ -13,6 +13,7 @@ struct config;
/**
* Opaque type
*/
struct ts2phc_private;
struct ts2phc_master;
/**
@ -26,12 +27,13 @@ enum ts2phc_master_type {
/**
* Create a new instance of a PPS master clock.
* @param cfg Pointer to a valid configuration.
* @param priv Pointer to the program's data structure.
* @param dev Name of the master clock or NULL.
* @param type The type of the clock to create.
* @return A pointer to a new PPS master clock on success, NULL otherwise.
*/
struct ts2phc_master *ts2phc_master_create(struct config *cfg, const char *dev,
struct ts2phc_master *ts2phc_master_create(struct ts2phc_private *priv,
const char *dev,
enum ts2phc_master_type type);
/**
@ -49,4 +51,6 @@ void ts2phc_master_destroy(struct ts2phc_master *master);
*/
int ts2phc_master_getppstime(struct ts2phc_master *master, struct timespec *ts);
struct clock *ts2phc_master_get_clock(struct ts2phc_master *m);
#endif

View File

@ -15,6 +15,7 @@
struct ts2phc_master {
void (*destroy)(struct ts2phc_master *ts2phc_master);
int (*getppstime)(struct ts2phc_master *master, struct timespec *ts);
struct clock *(*get_clock)(struct ts2phc_master *m);
};
#endif

View File

@ -198,7 +198,8 @@ static int ts2phc_nmea_master_getppstime(struct ts2phc_master *master,
return fix_valid ? lstab_error : -1;
}
struct ts2phc_master *ts2phc_nmea_master_create(struct config *cfg, const char *dev)
struct ts2phc_master *ts2phc_nmea_master_create(struct ts2phc_private *priv,
const char *dev)
{
struct ts2phc_nmea_master *master;
const char *leapfile = NULL; // TODO - read from config.
@ -214,7 +215,7 @@ struct ts2phc_master *ts2phc_nmea_master_create(struct config *cfg, const char *
}
master->master.destroy = ts2phc_nmea_master_destroy;
master->master.getppstime = ts2phc_nmea_master_getppstime;
master->config = cfg;
master->config = priv->cfg;
pthread_mutex_init(&master->mutex, NULL);
err = pthread_create(&master->worker, NULL, monitor_nmea_status, master);
if (err) {

View File

@ -6,8 +6,9 @@
#ifndef HAVE_TS2PHC_NMEA_MASTER_H
#define HAVE_TS2PHC_NMEA_MASTER_H
#include "ts2phc.h"
#include "ts2phc_master.h"
struct ts2phc_master *ts2phc_nmea_master_create(struct config *cfg,
struct ts2phc_master *ts2phc_nmea_master_create(struct ts2phc_private *priv,
const char *dev);
#endif

View File

@ -12,15 +12,14 @@
#include "phc.h"
#include "print.h"
#include "missing.h"
#include "ts2phc.h"
#include "ts2phc_master_private.h"
#include "ts2phc_phc_master.h"
#include "util.h"
struct ts2phc_phc_master {
struct ts2phc_master master;
clockid_t clkid;
struct clock *clock;
int channel;
int fd;
};
static int ts2phc_phc_master_activate(struct config *cfg, const char *dev,
@ -28,6 +27,8 @@ static int ts2phc_phc_master_activate(struct config *cfg, const char *dev,
{
struct ptp_perout_request perout_request;
struct ptp_pin_desc desc;
int32_t perout_phase;
int32_t pulsewidth;
struct timespec ts;
memset(&desc, 0, sizeof(desc));
@ -38,21 +39,36 @@ static int ts2phc_phc_master_activate(struct config *cfg, const char *dev,
desc.func = PTP_PF_PEROUT;
desc.chan = master->channel;
if (phc_pin_setfunc(master->clkid, &desc)) {
if (phc_pin_setfunc(master->clock->clkid, &desc)) {
pr_warning("Failed to set the pin. Continuing bravely on...");
}
if (clock_gettime(master->clkid, &ts)) {
if (clock_gettime(master->clock->clkid, &ts)) {
perror("clock_gettime");
return -1;
}
perout_phase = config_get_int(cfg, dev, "ts2phc.perout_phase");
memset(&perout_request, 0, sizeof(perout_request));
perout_request.index = master->channel;
perout_request.start.sec = ts.tv_sec + 2;
perout_request.start.nsec = 0;
perout_request.period.sec = 1;
perout_request.period.nsec = 0;
perout_request.flags = 0;
pulsewidth = config_get_int(cfg, dev, "ts2phc.pulsewidth");
if (pulsewidth) {
perout_request.flags |= PTP_PEROUT_DUTY_CYCLE;
perout_request.on.sec = pulsewidth / NS_PER_SEC;
perout_request.on.nsec = pulsewidth % NS_PER_SEC;
}
if (perout_phase != -1) {
perout_request.flags |= PTP_PEROUT_PHASE;
perout_request.phase.sec = perout_phase / NS_PER_SEC;
perout_request.phase.nsec = perout_phase % NS_PER_SEC;
} else {
perout_request.start.sec = ts.tv_sec + 2;
perout_request.start.nsec = 0;
}
if (ioctl(master->fd, PTP_PEROUT_REQUEST2, &perout_request)) {
if (ioctl(CLOCKID_TO_FD(master->clock->clkid), PTP_PEROUT_REQUEST2,
&perout_request)) {
pr_err(PTP_PEROUT_REQUEST_FAILED);
return -1;
}
@ -67,10 +83,11 @@ static void ts2phc_phc_master_destroy(struct ts2phc_master *master)
memset(&perout_request, 0, sizeof(perout_request));
perout_request.index = m->channel;
if (ioctl(m->fd, PTP_PEROUT_REQUEST2, &perout_request)) {
if (ioctl(CLOCKID_TO_FD(m->clock->clkid), PTP_PEROUT_REQUEST2,
&perout_request)) {
pr_err(PTP_PEROUT_REQUEST_FAILED);
}
posix_clock_close(m->clkid);
clock_destroy(m->clock);
free(m);
}
@ -79,14 +96,21 @@ static int ts2phc_phc_master_getppstime(struct ts2phc_master *m,
{
struct ts2phc_phc_master *master =
container_of(m, struct ts2phc_phc_master, master);
return clock_gettime(master->clkid, ts);
return clock_gettime(master->clock->clkid, ts);
}
struct ts2phc_master *ts2phc_phc_master_create(struct config *cfg,
struct clock *ts2phc_phc_master_get_clock(struct ts2phc_master *m)
{
struct ts2phc_phc_master *master =
container_of(m, struct ts2phc_phc_master, master);
return master->clock;
}
struct ts2phc_master *ts2phc_phc_master_create(struct ts2phc_private *priv,
const char *dev)
{
struct ts2phc_phc_master *master;
int junk;
master = calloc(1, sizeof(*master));
if (!master) {
@ -94,17 +118,19 @@ struct ts2phc_master *ts2phc_phc_master_create(struct config *cfg,
}
master->master.destroy = ts2phc_phc_master_destroy;
master->master.getppstime = ts2phc_phc_master_getppstime;
master->master.get_clock = ts2phc_phc_master_get_clock;
master->clkid = posix_clock_open(dev, &junk);
if (master->clkid == CLOCK_INVALID) {
master->clock = clock_add(priv, dev);
if (!master->clock) {
free(master);
return NULL;
}
master->fd = CLOCKID_TO_FD(master->clkid);
master->clock->is_destination = 0;
pr_debug("PHC master %s has ptp index %d", dev, junk);
pr_debug("PHC master %s has ptp index %d", dev,
master->clock->phc_index);
if (ts2phc_phc_master_activate(cfg, dev, master)) {
if (ts2phc_phc_master_activate(priv->cfg, dev, master)) {
ts2phc_phc_master_destroy(&master->master);
return NULL;
}

View File

@ -6,9 +6,10 @@
#ifndef HAVE_TS2PHC_PHC_MASTER_H
#define HAVE_TS2PHC_PHC_MASTER_H
#include "ts2phc.h"
#include "ts2phc_master.h"
struct ts2phc_master *ts2phc_phc_master_create(struct config *cfg,
struct ts2phc_master *ts2phc_phc_master_create(struct ts2phc_private *priv,
const char *dev);
#endif

View File

@ -15,43 +15,30 @@
#include <time.h>
#include <unistd.h>
#include "config.h"
#include "clockadj.h"
#include "config.h"
#include "missing.h"
#include "phc.h"
#include "print.h"
#include "servo.h"
#include "ts2phc_master.h"
#include "ts2phc_slave.h"
#include "ts2phc.h"
#include "util.h"
#define NS_PER_SEC 1000000000LL
#define SAMPLE_WEIGHT 1.0
#define SERVO_SYNC_INTERVAL 1.0
struct ts2phc_slave {
char *name;
STAILQ_ENTRY(ts2phc_slave) list;
struct ptp_pin_desc pin_desc;
enum servo_state state;
unsigned int polarity;
int32_t correction;
tmv_t correction;
uint32_t ignore_lower;
uint32_t ignore_upper;
struct servo *servo;
clockid_t clk;
int no_adj;
int fd;
struct clock *clock;
};
struct ts2phc_slave_array {
struct ts2phc_slave **slave;
int *collected_events;
struct pollfd *pfd;
} polling_array;
struct ts2phc_source_timestamp {
struct timespec ts;
bool valid;
};
enum extts_result {
@ -60,61 +47,77 @@ enum extts_result {
EXTTS_IGNORE = 1,
};
static enum extts_result ts2phc_slave_offset(struct ts2phc_slave *slave,
struct ts2phc_source_timestamp ts,
int64_t *offset,
uint64_t *local_ts);
static STAILQ_HEAD(slave_ifaces_head, ts2phc_slave) ts2phc_slaves =
STAILQ_HEAD_INITIALIZER(ts2phc_slaves);
static unsigned int ts2phc_n_slaves;
static int ts2phc_slave_array_create(void)
static int ts2phc_slave_array_create(struct ts2phc_private *priv)
{
struct ts2phc_slave_array *polling_array;
struct ts2phc_slave *slave;
unsigned int i;
if (polling_array.slave) {
return 0;
}
polling_array.slave = malloc(ts2phc_n_slaves * sizeof(*polling_array.slave));
if (!polling_array.slave) {
polling_array = malloc(sizeof(*polling_array));
if (!polling_array) {
pr_err("low memory");
return -1;
}
polling_array.pfd = malloc(ts2phc_n_slaves * sizeof(*polling_array.pfd));
if (!polling_array.pfd) {
polling_array->slave = malloc(priv->n_slaves *
sizeof(*polling_array->slave));
if (!polling_array->slave) {
pr_err("low memory");
free(polling_array.slave);
polling_array.slave = NULL;
return -1;
}
polling_array->pfd = malloc(priv->n_slaves *
sizeof(*polling_array->pfd));
if (!polling_array->pfd) {
pr_err("low memory");
free(polling_array->slave);
polling_array->slave = NULL;
return -1;
}
polling_array->collected_events = malloc(priv->n_slaves * sizeof(int));
if (!polling_array->collected_events) {
pr_err("low memory");
free(polling_array->slave);
free(polling_array->pfd);
polling_array->pfd = NULL;
polling_array->slave = NULL;
return -1;
}
i = 0;
STAILQ_FOREACH(slave, &ts2phc_slaves, list) {
polling_array.slave[i] = slave;
STAILQ_FOREACH(slave, &priv->slaves, list) {
polling_array->slave[i] = slave;
i++;
}
for (i = 0; i < ts2phc_n_slaves; i++) {
polling_array.pfd[i].events = POLLIN | POLLPRI;
polling_array.pfd[i].fd = polling_array.slave[i]->fd;
for (i = 0; i < priv->n_slaves; i++) {
struct ts2phc_slave *slave = polling_array->slave[i];
polling_array->pfd[i].events = POLLIN | POLLPRI;
polling_array->pfd[i].fd = CLOCKID_TO_FD(slave->clock->clkid);
}
priv->polling_array = polling_array;
return 0;
}
static void ts2phc_slave_array_destroy(void)
static void ts2phc_slave_array_destroy(struct ts2phc_private *priv)
{
free(polling_array.slave);
free(polling_array.pfd);
polling_array.slave = NULL;
polling_array.pfd = NULL;
struct ts2phc_slave_array *polling_array = priv->polling_array;
if (!polling_array)
return;
free(polling_array->slave);
free(polling_array->pfd);
free(polling_array->collected_events);
free(polling_array);
priv->polling_array = NULL;
}
static int ts2phc_slave_clear_fifo(struct ts2phc_slave *slave)
{
struct pollfd pfd = {
.events = POLLIN | POLLPRI,
.fd = slave->fd,
.fd = CLOCKID_TO_FD(slave->clock->clkid),
};
struct ptp_extts_event event;
int cnt, size;
@ -143,12 +146,13 @@ static int ts2phc_slave_clear_fifo(struct ts2phc_slave *slave)
return 0;
}
static struct ts2phc_slave *ts2phc_slave_create(struct config *cfg, const char *device)
static struct ts2phc_slave *ts2phc_slave_create(struct ts2phc_private *priv,
const char *device)
{
enum servo_type servo = config_get_int(cfg, NULL, "clock_servo");
int err, fadj, junk, max_adj, pulsewidth;
struct ptp_extts_request extts;
struct ts2phc_slave *slave;
int err, pulsewidth;
int32_t correction;
slave = calloc(1, sizeof(*slave));
if (!slave) {
@ -161,44 +165,35 @@ static struct ts2phc_slave *ts2phc_slave_create(struct config *cfg, const char *
free(slave);
return NULL;
}
slave->pin_desc.index = config_get_int(cfg, device, "ts2phc.pin_index");
slave->pin_desc.index = config_get_int(priv->cfg, device,
"ts2phc.pin_index");
slave->pin_desc.func = PTP_PF_EXTTS;
slave->pin_desc.chan = config_get_int(cfg, device, "ts2phc.channel");
slave->polarity = config_get_int(cfg, device, "ts2phc.extts_polarity");
slave->correction = config_get_int(cfg, device, "ts2phc.extts_correction");
slave->pin_desc.chan = config_get_int(priv->cfg, device,
"ts2phc.channel");
slave->polarity = config_get_int(priv->cfg, device,
"ts2phc.extts_polarity");
correction = config_get_int(priv->cfg, device,
"ts2phc.extts_correction");
slave->correction = nanoseconds_to_tmv(correction);
pulsewidth = config_get_int(cfg, device, "ts2phc.pulsewidth");
pulsewidth = config_get_int(priv->cfg, device,
"ts2phc.pulsewidth");
pulsewidth /= 2;
slave->ignore_upper = 1000000000 - pulsewidth;
slave->ignore_lower = pulsewidth;
slave->clk = posix_clock_open(device, &junk);
if (slave->clk == CLOCK_INVALID) {
slave->clock = clock_add(priv, device);
if (!slave->clock) {
pr_err("failed to open clock");
goto no_posix_clock;
}
slave->no_adj = config_get_int(cfg, NULL, "free_running");
slave->fd = CLOCKID_TO_FD(slave->clk);
slave->clock->is_destination = 1;
pr_debug("PHC slave %s has ptp index %d", device, junk);
pr_debug("PHC slave %s has ptp index %d", device,
slave->clock->phc_index);
fadj = (int) clockadj_get_freq(slave->clk);
/* Due to a bug in older kernels, the reading may silently fail
and return 0. Set the frequency back to make sure fadj is
the actual frequency of the clock. */
clockadj_set_freq(slave->clk, fadj);
max_adj = phc_max_adj(slave->clk);
slave->servo = servo_create(cfg, servo, -fadj, max_adj, 0);
if (!slave->servo) {
pr_err("failed to create servo");
goto no_servo;
}
servo_sync_interval(slave->servo, SERVO_SYNC_INTERVAL);
if (phc_number_pins(slave->clk) > 0) {
err = phc_pin_setfunc(slave->clk, &slave->pin_desc);
if (phc_number_pins(slave->clock->clkid) > 0) {
err = phc_pin_setfunc(slave->clock->clkid, &slave->pin_desc);
if (err < 0) {
pr_err("PTP_PIN_SETFUNC request failed");
goto no_pin_func;
@ -212,7 +207,8 @@ static struct ts2phc_slave *ts2phc_slave_create(struct config *cfg, const char *
memset(&extts, 0, sizeof(extts));
extts.index = slave->pin_desc.chan;
extts.flags = 0;
if (ioctl(slave->fd, PTP_EXTTS_REQUEST2, &extts)) {
if (ioctl(CLOCKID_TO_FD(slave->clock->clkid), PTP_EXTTS_REQUEST2,
&extts)) {
pr_err(PTP_EXTTS_REQUEST_FAILED);
}
if (ts2phc_slave_clear_fifo(slave)) {
@ -222,9 +218,7 @@ static struct ts2phc_slave *ts2phc_slave_create(struct config *cfg, const char *
return slave;
no_ext_ts:
no_pin_func:
servo_destroy(slave->servo);
no_servo:
posix_clock_close(slave->clk);
clock_destroy(slave->clock);
no_posix_clock:
free(slave->name);
free(slave);
@ -238,136 +232,101 @@ static void ts2phc_slave_destroy(struct ts2phc_slave *slave)
memset(&extts, 0, sizeof(extts));
extts.index = slave->pin_desc.chan;
extts.flags = 0;
if (ioctl(slave->fd, PTP_EXTTS_REQUEST2, &extts)) {
if (ioctl(CLOCKID_TO_FD(slave->clock->clkid), PTP_EXTTS_REQUEST2,
&extts)) {
pr_err(PTP_EXTTS_REQUEST_FAILED);
}
servo_destroy(slave->servo);
posix_clock_close(slave->clk);
clock_destroy(slave->clock);
free(slave->name);
free(slave);
}
static int ts2phc_slave_event(struct ts2phc_slave *slave,
struct ts2phc_source_timestamp source_ts)
static bool ts2phc_slave_ignore(struct ts2phc_private *priv,
struct ts2phc_slave *slave,
struct timespec source_ts)
{
enum extts_result result;
uint64_t extts_ts;
int64_t offset;
double adj;
tmv_t source_tmv = timespec_to_tmv(source_ts);
result = ts2phc_slave_offset(slave, source_ts, &offset, &extts_ts);
switch (result) {
case EXTTS_ERROR:
return -1;
case EXTTS_OK:
break;
case EXTTS_IGNORE:
return 0;
}
source_tmv = tmv_sub(source_tmv, priv->perout_phase);
source_ts = tmv_to_timespec(source_tmv);
if (slave->no_adj) {
pr_info("%s master offset %10" PRId64, slave->name, offset);
return 0;
}
if (!source_ts.valid) {
pr_debug("%s ignoring invalid master time stamp", slave->name);
return 0;
}
adj = servo_sample(slave->servo, offset, extts_ts,
SAMPLE_WEIGHT, &slave->state);
pr_debug("%s master offset %10" PRId64 " s%d freq %+7.0f",
slave->name, offset, slave->state, adj);
switch (slave->state) {
case SERVO_UNLOCKED:
break;
case SERVO_JUMP:
clockadj_set_freq(slave->clk, -adj);
clockadj_step(slave->clk, -offset);
break;
case SERVO_LOCKED:
case SERVO_LOCKED_STABLE:
clockadj_set_freq(slave->clk, -adj);
break;
}
return 0;
return source_ts.tv_nsec > slave->ignore_lower &&
source_ts.tv_nsec < slave->ignore_upper;
}
static enum extts_result ts2phc_slave_offset(struct ts2phc_slave *slave,
struct ts2phc_source_timestamp src,
int64_t *offset,
uint64_t *local_ts)
static enum extts_result ts2phc_slave_event(struct ts2phc_private *priv,
struct ts2phc_slave *slave)
{
struct timespec source_ts = src.ts;
enum extts_result result = EXTTS_OK;
struct ptp_extts_event event;
uint64_t event_ns, source_ns;
int cnt;
struct timespec source_ts;
int err, cnt;
tmv_t ts;
cnt = read(slave->fd, &event, sizeof(event));
cnt = read(CLOCKID_TO_FD(slave->clock->clkid), &event, sizeof(event));
if (cnt != sizeof(event)) {
pr_err("read extts event failed: %m");
return EXTTS_ERROR;
result = EXTTS_ERROR;
goto out;
}
if (event.index != slave->pin_desc.chan) {
pr_err("extts on unexpected channel");
return EXTTS_ERROR;
result = EXTTS_ERROR;
goto out;
}
err = ts2phc_master_getppstime(priv->master, &source_ts);
if (err < 0) {
pr_debug("source ts not valid");
return 0;
}
event_ns = event.t.sec * NS_PER_SEC;
event_ns += event.t.nsec;
if (slave->polarity == (PTP_RISING_EDGE | PTP_FALLING_EDGE) &&
source_ts.tv_nsec > slave->ignore_lower &&
source_ts.tv_nsec < slave->ignore_upper) {
ts2phc_slave_ignore(priv, slave, source_ts)) {
pr_debug("%s SKIP extts index %u at %lld.%09u src %" PRIi64 ".%ld",
slave->name, event.index, event.t.sec, event.t.nsec,
(int64_t) source_ts.tv_sec, source_ts.tv_nsec);
return EXTTS_IGNORE;
result = EXTTS_IGNORE;
goto out;
}
if (source_ts.tv_nsec > 500000000) {
source_ts.tv_sec++;
}
source_ns = source_ts.tv_sec * NS_PER_SEC;
*offset = event_ns + slave->correction - source_ns;
*local_ts = event_ns + slave->correction;
pr_debug("%s extts index %u at %lld.%09u corr %d src %" PRIi64
".%ld diff %" PRId64,
slave->name, event.index, event.t.sec, event.t.nsec,
slave->correction,
(int64_t) source_ts.tv_sec, source_ts.tv_nsec, *offset);
out:
if (result == EXTTS_ERROR || result == EXTTS_IGNORE)
return result;
ts = pct_to_tmv(event.t);
ts = tmv_add(ts, slave->correction);
clock_add_tstamp(slave->clock, ts);
return EXTTS_OK;
}
/* public methods */
int ts2phc_slave_add(struct config *cfg, const char *name)
int ts2phc_slave_add(struct ts2phc_private *priv, const char *name)
{
struct ts2phc_slave *slave;
/* Create each interface only once. */
STAILQ_FOREACH(slave, &ts2phc_slaves, list) {
STAILQ_FOREACH(slave, &priv->slaves, list) {
if (0 == strcmp(name, slave->name)) {
return 0;
}
}
slave = ts2phc_slave_create(cfg, name);
slave = ts2phc_slave_create(priv, name);
if (!slave) {
pr_err("failed to create slave");
return -1;
}
STAILQ_INSERT_TAIL(&ts2phc_slaves, slave, list);
ts2phc_n_slaves++;
STAILQ_INSERT_TAIL(&priv->slaves, slave, list);
priv->n_slaves++;
return 0;
}
int ts2phc_slave_arm(void)
int ts2phc_slave_arm(struct ts2phc_private *priv)
{
struct ptp_extts_request extts;
struct ts2phc_slave *slave;
@ -375,10 +334,11 @@ int ts2phc_slave_arm(void)
memset(&extts, 0, sizeof(extts));
STAILQ_FOREACH(slave, &ts2phc_slaves, list) {
STAILQ_FOREACH(slave, &priv->slaves, list) {
extts.index = slave->pin_desc.chan;
extts.flags = slave->polarity | PTP_ENABLE_FEATURE;
err = ioctl(slave->fd, PTP_EXTTS_REQUEST2, &extts);
err = ioctl(CLOCKID_TO_FD(slave->clock->clkid),
PTP_EXTTS_REQUEST2, &extts);
if (err < 0) {
pr_err(PTP_EXTTS_REQUEST_FAILED);
return -1;
@ -387,48 +347,91 @@ int ts2phc_slave_arm(void)
return 0;
}
void ts2phc_slave_cleanup(void)
int ts2phc_slaves_init(struct ts2phc_private *priv)
{
int err;
err = ts2phc_slave_array_create(priv);
if (err)
return err;
return ts2phc_slave_arm(priv);
}
void ts2phc_slave_cleanup(struct ts2phc_private *priv)
{
struct ts2phc_slave *slave;
ts2phc_slave_array_destroy();
ts2phc_slave_array_destroy(priv);
while ((slave = STAILQ_FIRST(&ts2phc_slaves))) {
STAILQ_REMOVE_HEAD(&ts2phc_slaves, list);
while ((slave = STAILQ_FIRST(&priv->slaves))) {
STAILQ_REMOVE_HEAD(&priv->slaves, list);
ts2phc_slave_destroy(slave);
ts2phc_n_slaves--;
priv->n_slaves--;
}
}
int ts2phc_slave_poll(struct ts2phc_master *master)
int ts2phc_slave_poll(struct ts2phc_private *priv)
{
struct ts2phc_source_timestamp source_ts;
struct ts2phc_slave_array *polling_array = priv->polling_array;
int all_slaves_have_events = 0;
int ignore_any = 0;
unsigned int i;
int cnt, err;
int cnt;
if (ts2phc_slave_array_create()) {
return -1;
}
cnt = poll(polling_array.pfd, ts2phc_n_slaves, 2000);
if (cnt < 0) {
if (EINTR == errno) {
for (i = 0; i < priv->n_slaves; i++)
polling_array->collected_events[i] = 0;
while (!all_slaves_have_events) {
struct ts2phc_slave *slave;
cnt = poll(polling_array->pfd, priv->n_slaves, 2000);
if (cnt < 0) {
if (EINTR == errno) {
return 0;
} else {
pr_emerg("poll failed");
return -1;
}
} else if (!cnt) {
pr_debug("poll returns zero, no events");
return 0;
} else {
pr_emerg("poll failed");
return -1;
}
} else if (!cnt) {
pr_debug("poll returns zero, no events");
for (i = 0; i < priv->n_slaves; i++) {
if (polling_array->pfd[i].revents & (POLLIN|POLLPRI)) {
enum extts_result result;
slave = polling_array->slave[i];
result = ts2phc_slave_event(priv, slave);
if (result == EXTTS_ERROR)
return -EIO;
if (result == EXTTS_IGNORE)
ignore_any = 1;
/*
* Collect the events anyway, even if we'll
* ignore this master edge anyway. We don't
* want slave events from different edges
* to pile up and mix.
*/
polling_array->collected_events[i]++;
}
}
all_slaves_have_events = true;
for (i = 0; i < priv->n_slaves; i++) {
if (!polling_array->collected_events[i]) {
all_slaves_have_events = false;
break;
}
}
}
if (ignore_any)
return 0;
}
err = ts2phc_master_getppstime(master, &source_ts.ts);
source_ts.valid = err ? false : true;
for (i = 0; i < ts2phc_n_slaves; i++) {
if (polling_array.pfd[i].revents & (POLLIN|POLLPRI)) {
ts2phc_slave_event(polling_array.slave[i], source_ts);
}
}
return 0;
return 1;
}

View File

@ -7,14 +7,14 @@
#ifndef HAVE_TS2PHC_SLAVE_H
#define HAVE_TS2PHC_SLAVE_H
#include "ts2phc_master.h"
#include "ts2phc.h"
int ts2phc_slave_add(struct config *cfg, const char *name);
int ts2phc_slave_add(struct ts2phc_private *priv, const char *name);
int ts2phc_slave_arm(void);
int ts2phc_slaves_init(struct ts2phc_private *priv);
void ts2phc_slave_cleanup(void);
void ts2phc_slave_cleanup(struct ts2phc_private *priv);
int ts2phc_slave_poll(struct ts2phc_master *master);
int ts2phc_slave_poll(struct ts2phc_private *priv);
#endif