All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session
@ 2020-10-09 14:03 Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization Tzvetomir Stoyanov (VMware)
                   ` (9 more replies)
  0 siblings, 10 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

Basic infrastructure for host - guest timestamp synchronization and a
POC implementation of PTP-like and KVM algorithms. 

The KVM plugin is with higher weight than PTP, it is prefered in case both
are active. KVM is used only if x86-tsc clock is selected for the current
trace, PTP is used for all other cases.

[
 v24 changes:
  - trace-cmd internal refactoring, needed for KVM timestamp synchronization
    plugin.
  - Added KVM timestamp synchronization plugin.

 v23 changes:
  - Added flags to PTP start message to control the behaviour of the
    algorithm. This is for development purposes mostly, to compare and
    evaluate how these changes affect the accuracy.
	PTP_FLAG_FASTEST_RESPONSE:  Consider only the probe with fastest
				    response time, otherwise make a histogram
				    from all probes.
	PTP_FLAG_USE_MARKER: Use trace marker to get the clock,
 			     otherwise use directly the system clock

 v22 changes, addressed Steven's comments:
  - Fixed error flow in read_qemu_guests_pids(), free allocated resources
    in case of an error.
  - Fixed initialisation of reallocated memmory in set_vcpu_pid_mapping().
  - Fixed bitmask logic in tracecmd_tsync_proto_select().
  - Fixed usage of wrong structure in make_trace_resp().

 v21 changes:
  - Rebased on top of latest master.
  - Remove these two patches from the set, as they are already merged:
       trace-cmd: Add new library API for local CPU count
       trace-cmd: Add support for negative time offsets in trace.dat file
  - Added more descriptive warning message when fails to extract Time Shift
    information from the trace.dat file.
  - Added a warning message when fails to obtain vcpu - pid mappings.
  - Handle the case with sparse VCPU numbers in VCPU - PID mapping
    array.
  - Fixed broken recording over FIFOs.
 v20 changes:
  - Rebased on top of latest master.
  - Removed the VCPUS_MAX hardcoded limit, reimplemented the cpu_pid[] array
    to be dynamically allocated.
  - Added a warning when reading of TRACECMD_OPTION_TIME_SHIFT option
    fails, due to unexpected option's size.
  - Improved loading of TRACECMD_OPTION_GUEST option data, as Steven
    suggested.
 v19 changes:
  - Rebased on top of latest master. The librtacefs is already merged,
    this allows to drop few patches from the set and use librtacefs APIs
    instead.
  - Reimplemented all new trace.dat options to be in binary format,
    instead of text. Leverage the new "trace-cmd dump" sub command to
    track what is written in the file.
  - Addressed Steven's comments.

 v18 changes: addressed Steven Rostdet comments:
  - Replaced semaphores with pthread mutexes.
  - Made bitmask with time sync protocols unlimited, so we can support
    more than 32 protocols. That required to redesign the trace request
    packet format.
  - A lot of small fixes.
	
 v17 changes:
  - Implemented new PTP logic for calculating the clocks offset, using
    histogram of all PTP samples. It gives better results than the logic
    with the fastest response time, so set the new one as default in the
    POC patch.

 v16 changes:
  - Fixed compilation in case no VSOCK is available (Thanks to Slavomir Kaslev)
  - Fixed a typo in trace-cmd-record.1.txt (Thanks to Slavomir Kaslev)
  - Added forgotten file in the patch "trace-cmd: Add new library APIs for
    ftrace instances." - trace-instance.c
  - Fixed few compilation warnings related to TSYNC_DEBUG code.
  - Removed a blank line at the end of "tsync_readme" file. 

 v15 changes:
  - Removed the patch for "--proc-map" from the series, as it should not
    be part of it.

 v14 changes:
  - Bring back the PTP-like algorithm and removed the ftrace event based logic.
  - Reimplemented the PTP-like algorithm to use raw ftrace markers, instead of clock_gettime() API.
  - Refactored the logic to be algorithm independent and plugin friendly.
  - Implemented continuous timestamps synchronization, while the trace is running.
  - Moved logic from trace-cmd application to libtracecmd, as new library APIs.
  - Implemented new trace id functionality.
  - Implemented new guest section in host trace.dat file.
	
 v13 changes:
  - Remove few patches from the set, as they were merged.
  - Rebased to the latest master, Slavomirs patchest "Add VM kernel tracing over
    vsockets and FIFOs" got merged!
 	
 v12 changes:
  - Rebased on top of Slavomir's v13 "Add VM kernel tracing over vsockets and FIFOs"

 v11 changes:
  - Rebased on top of Slavomir's v10 "Add VM kernel tracing over vsockets and FIFOs"
  - Addressed Slavomir's commnents from version 10 of the patch series.

 v10 changes:
  - Fixed broken compilation, call to timestamp_correction_calc() in timestamp_correct
    was smashed.
  - Replaced deprecated tep_data_event_from_type() API with tep_find_event().
  - Fixed a warning on assignment const to non const.

 v9 changes:
  - Fixed implementation of binary search algorithm in timestamp_correct()

 v8 changes:
  - Added rmdir() call in tracecmd_remove_instance(), to completely remove the instance. 
  However, there is an issue with deleting the instances using rmdir(), which is investigated.
  - Few changes in read_qemu_guests_pids(), timestamp_correct(), tsync_offset_load() 
 tracecmd_clock_context_new() and find_raw_events() suggested by Slavomir. 

 v7 changes:
  - Added warning messages in case time synchronization cannot be negotiated or fails.
  - Few optimizations and checks in read_qemu_guests_pids(), tsync_offset_load(),
    and find_raw_events(), suggested by Slavomir Kaslev.
  - Reworked timestamp_correct() to not use static variables.
  - Check TRACECMD_OPTION_TIME_SHIFT before reading time sync samples from the trace.dat file

 v6 changes:
  - Refactored tracecmd_msg_snd_time_sync() and tracecmd_msg_rcv_time_sync() functions:
    removed any time sync calculations logic as separate functions in trace-timesync.c file
  - Defined TSYNC_PROBE, TSYNC_REQ and TSYNC_RESP messages, in order to make the time sync
    protocol comprehensible.
  - Addressed Steven Rostedt comments.
  - Addressed Slavomir Kaslev commnets.

v5 changes:
  - Rebased to Slavomir's v8 "Add VM kernel tracing over vsockets and FIFOs"
    patch series.
  - Implemented an algorithm for time drift correction.
  - Addressed Slavomir's commnets.
  - Refactored the code: moved all time sync specific implementation in trace-timesync.c
  - Isolated all hardcoded event specific stuff in a structure, so it could be easily
    moved to external plugins.
  - Added a check for VSOCK support: do not perform vsock dependent time synchronisation
    in case there is no VSOCK support.

 v4 changes:
  - Removed the implementation of PTP-like algorithm. The current
    logic relies on matching time stamps of kvm_exit/virtio_transport_recv_pkt
    events on host to virtio_transport_alloc_pkt/vp_notify events on guest.
  - Rebased to Slavomir's v7 "Add VM kernel tracing over vsockets and FIFOs"
    patch series.
  - Decreased the time synch probes from 5000 to 300.
  - Addressed Steven Rostedt comments.
  - Code cleanup.

 v3 changes:
 - Removed any magic constants, used in the PTP-like algorithm,
   as Slavomir Kaslev suggested.
 - Implemented new algorithm, based on mapping kvm_exit events
   in host context to vsock_send events in guest context,
   suggested by Steven Rostedt.

 v2 changes:
  - Addressed Steven Rostedt comments.
  - Modified PTP-like timestamps sync algorithm to gain more accuracy, with the
    help of Yordan Karadzhov and Slavomir Kaslev.
]

Tzvetomir Stoyanov (VMware) (10):
  trace-cmd: [POC] PTP-like algorithm for host - guest timestamp
    synchronization
  trace-cmd: Debug scripts for PTP-like algorithm for host - guest
    timestamp synchronization
  trace-cmd: Add trace-cmd library APIs for ftrace clock name
  trace-cmd: Move VM related logic in a separate file
  trace-cmd: Add clock parameter to timestamp synchronization plugins
  trace-cmd: Add role parameter to timestamp synchronization plugins
  trace-cmd: Add host / guest role in timestamp synchronization context
  trace-cmd: Add guest CPU count PID in tracecmd_time_sync struct
  trace-cmd: Fixed bitmask logic tracecmd_tsync_proto_getall()
  trace-cmd [POC]: Add KVM timestamp synchronization plugin

 include/trace-cmd/trace-cmd.h             |  33 +-
 lib/trace-cmd/Makefile                    |   2 +
 lib/trace-cmd/include/trace-tsync-local.h |  13 +-
 lib/trace-cmd/trace-timesync-kvm.c        | 427 +++++++++++++
 lib/trace-cmd/trace-timesync-ptp.c        | 693 ++++++++++++++++++++++
 lib/trace-cmd/trace-timesync.c            |  59 +-
 lib/trace-cmd/trace-util.c                |  45 ++
 scripts/debug/tsync_hist.py               |  57 ++
 scripts/debug/tsync_readme                |  12 +
 scripts/debug/tsync_res.py                |  46 ++
 tracecmd/Makefile                         |   1 +
 tracecmd/include/trace-local.h            |  20 +
 tracecmd/trace-agent.c                    |   2 +
 tracecmd/trace-record.c                   | 260 ++------
 tracecmd/trace-tsync.c                    |  10 +-
 tracecmd/trace-vm.c                       | 214 +++++++
 16 files changed, 1648 insertions(+), 246 deletions(-)
 create mode 100644 lib/trace-cmd/trace-timesync-kvm.c
 create mode 100644 lib/trace-cmd/trace-timesync-ptp.c
 create mode 100644 scripts/debug/tsync_hist.py
 create mode 100644 scripts/debug/tsync_readme
 create mode 100644 scripts/debug/tsync_res.py
 create mode 100644 tracecmd/trace-vm.c

-- 
2.26.2


^ permalink raw reply	[flat|nested] 16+ messages in thread

* [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-15 21:24   ` Steven Rostedt
  2020-10-09 14:03 ` [PATCH v24 02/10] trace-cmd: Debug scripts for " Tzvetomir Stoyanov (VMware)
                   ` (8 subsequent siblings)
  9 siblings, 1 reply; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

PTP protocol is designed for synchronizing clocks of machines in a local network.
The same approach can be used for host - guest timestamp synchronization.
This implementation uses ftrace raw markers to track trace timestamps of PTP events.
The patch is a POC, two different algorithms for PTP calculations are proposed:
  - Choosing the sample with the fastest response time for calculating the clocks offset.
  - Calculating the clocks offset using the average of all PTP samples.

The implementation can be tuned using those parameters:
 - #define FASTEST_RESPONSE - is defined, the sample with the fastest response time
    is used for calculating the clocks offset. Otherwise the histogram of all samples is used.
 - #define PTP_SYNC_LOOP 340 - defines the number of samples, used for one calculation.
 - --tsync-interval - a trace-cmd argument, choose the intervals between offset calculations,
	performed continuously during the trace.
 - #define TSYNC_DEBUG - if defined, a debug information is collected and stored in files,
   in the guest machine:
     s-cid*.txt - For each offset calculation: host and guest clocks and calculated offset.
     res-cid*.txt - For each tracing session: all calculated clock offsets.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 include/trace-cmd/trace-cmd.h      |   1 +
 lib/trace-cmd/Makefile             |   1 +
 lib/trace-cmd/trace-timesync-ptp.c | 690 +++++++++++++++++++++++++++++
 lib/trace-cmd/trace-timesync.c     |   8 +
 tracecmd/trace-agent.c             |   2 +
 tracecmd/trace-record.c            |  23 +-
 6 files changed, 717 insertions(+), 8 deletions(-)
 create mode 100644 lib/trace-cmd/trace-timesync-ptp.c

diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
index f3c95f30..f9c1f843 100644
--- a/include/trace-cmd/trace-cmd.h
+++ b/include/trace-cmd/trace-cmd.h
@@ -419,6 +419,7 @@ int tracecmd_msg_recv_time_sync(struct tracecmd_msg_handle *msg_handle,
 
 enum{
 	TRACECMD_TIME_SYNC_PROTO_NONE	= 0,
+	TRACECMD_TIME_SYNC_PROTO_PTP	= 1,
 };
 enum{
 	TRACECMD_TIME_SYNC_CMD_PROBE	= 1,
diff --git a/lib/trace-cmd/Makefile b/lib/trace-cmd/Makefile
index 666a1ebf..7cab7514 100644
--- a/lib/trace-cmd/Makefile
+++ b/lib/trace-cmd/Makefile
@@ -18,6 +18,7 @@ OBJS += trace-msg.o
 OBJS += trace-plugin.o
 ifeq ($(VSOCK_DEFINED), 1)
 OBJS += trace-timesync.o
+OBJS += trace-timesync-ptp.o
 endif
 
 # Additional util objects
diff --git a/lib/trace-cmd/trace-timesync-ptp.c b/lib/trace-cmd/trace-timesync-ptp.c
new file mode 100644
index 00000000..19e2f647
--- /dev/null
+++ b/lib/trace-cmd/trace-timesync-ptp.c
@@ -0,0 +1,690 @@
+// SPDX-License-Identifier: LGPL-2.1
+/*
+ * Copyright (C) 2019, VMware, Tzvetomir Stoyanov tz.stoyanov@gmail.com>
+ *
+ */
+
+#include <fcntl.h>
+#include <stdlib.h>
+#include <unistd.h>
+#include <arpa/inet.h>
+#include <linux/vm_sockets.h>
+#include <sys/types.h>
+#include <linux/types.h>
+#include <time.h>
+#include <sched.h>
+#include "trace-cmd.h"
+#include "tracefs.h"
+#include "trace-tsync-local.h"
+#include "trace-msg.h"
+#include "trace-cmd-local.h"
+
+typedef __be32 be32;
+typedef __u64 u64;
+typedef __s64 s64;
+
+#define PTP_SYNC_LOOP	340
+
+#define PTP_SYNC_PKT_START	1
+#define PTP_SYNC_PKT_PROBE	2
+#define PTP_SYNC_PKT_PROBES	3
+#define PTP_SYNC_PKT_OFFSET	4
+#define PTP_SYNC_PKT_END	5
+
+/* print time sync debug messages */
+#define TSYNC_DEBUG
+
+struct ptp_clock_sync {
+	struct tep_handle	*tep;
+	int			raw_id;
+	int			marker_fd;
+	int			series_id;
+	int			flags;
+	int			debug_fd;
+};
+
+enum {
+/* Consider only the probe with fastest response time,
+ * otherwise make a histogram from all probes.
+ */
+	PTP_FLAG_FASTEST_RESPONSE		= (1 << 0),
+/* Use trace marker to get the clock,
+ * otherwise use directly the system clock
+ */
+	PTP_FLAG_USE_MARKER			= (1 << 1),
+};
+static int ptp_flags = PTP_FLAG_FASTEST_RESPONSE | PTP_FLAG_USE_MARKER;
+
+struct ptp_clock_start_msg {
+	be32	series_id;
+	be32	flags;
+} __attribute__((packed));
+
+struct ptp_clock_sample {
+	s64		ts;
+	be32		id;
+} __attribute__((packed));
+
+struct ptp_clock_result_msg {
+	be32			series_id;
+	be32			count;
+	struct ptp_clock_sample	samples[2*PTP_SYNC_LOOP];
+} __attribute__((packed));
+
+struct ptp_clock_offset_msg {
+	s64	ts;
+	s64	offset;
+};
+
+struct ptp_markers_context {
+	struct clock_sync_context	*clock;
+	struct ptp_clock_sync		*ptp;
+	int				size;
+	struct ptp_clock_result_msg	msg;
+};
+
+struct ptp_marker_buf {
+	int local_cid;
+	int remote_cid;
+	int count;
+	int packet_id;
+} __attribute__((packed));
+
+struct ptp_marker {
+	int series_id;
+	struct ptp_marker_buf data;
+} __attribute__((packed));
+
+static int ptp_clock_sync_init(struct tracecmd_time_sync *tsync)
+{
+	const char *systems[] = {"ftrace", NULL};
+	struct clock_sync_context *clock_context;
+	struct ptp_clock_sync *ptp;
+	struct tep_event *raw;
+	char *path;
+
+	if (!tsync || !tsync->context)
+		return -1;
+	clock_context = (struct clock_sync_context *)tsync->context;
+	if (clock_context->proto_data)
+		return 0;
+
+	ptp = calloc(1, sizeof(struct ptp_clock_sync));
+	if (!ptp)
+		return -1;
+
+	ptp->marker_fd = -1;
+	ptp->debug_fd = -1;
+
+	path = tracefs_instance_get_dir(clock_context->instance);
+	if (!path)
+		goto error;
+	ptp->tep = tracefs_local_events_system(path, systems);
+	tracefs_put_tracing_file(path);
+	if (!ptp->tep)
+		goto error;
+	raw = tep_find_event_by_name(ptp->tep, "ftrace", "raw_data");
+	if (!raw)
+		goto error;
+
+	ptp->raw_id = raw->id;
+	tep_set_file_bigendian(ptp->tep, tracecmd_host_bigendian());
+	tep_set_local_bigendian(ptp->tep, tracecmd_host_bigendian());
+
+	path = tracefs_instance_get_file(clock_context->instance, "trace_marker_raw");
+	if (!path)
+		goto error;
+	ptp->marker_fd = open(path, O_WRONLY);
+	tracefs_put_tracing_file(path);
+
+	clock_context->proto_data = ptp;
+
+#ifdef TSYNC_DEBUG
+	if (clock_context->is_server) {
+		char buff[256];
+		int res_fd;
+
+		sprintf(buff, "res-cid%d.txt", clock_context->remote_cid);
+
+		res_fd = open(buff, O_CREAT|O_WRONLY|O_TRUNC, 0644);
+		if (res_fd > 0)
+			close(res_fd);
+	}
+#endif
+
+	return 0;
+
+error:
+	if (ptp) {
+		tep_free(ptp->tep);
+		if (ptp->marker_fd >= 0)
+			close(ptp->marker_fd);
+	}
+	free(ptp);
+	return -1;
+}
+
+static int ptp_clock_sync_free(struct tracecmd_time_sync *tsync)
+{
+	struct clock_sync_context *clock_context;
+	struct ptp_clock_sync *ptp;
+
+	if (!tsync || !tsync->context)
+		return -1;
+	clock_context = (struct clock_sync_context *)tsync->context;
+
+	if (clock_context && clock_context->proto_data) {
+		ptp = (struct ptp_clock_sync *)clock_context->proto_data;
+		tep_free(ptp->tep);
+		if (ptp->marker_fd >= 0)
+			close(ptp->marker_fd);
+		if (ptp->debug_fd >= 0)
+			close(ptp->debug_fd);
+		free(clock_context->proto_data);
+		clock_context->proto_data = NULL;
+	}
+	return 0;
+}
+
+static void ptp_probe_store(struct ptp_markers_context *ctx,
+			    struct ptp_marker *marker,
+			    unsigned long long ts)
+{
+	int index = -1;
+
+	if (marker->data.packet_id == 'r' &&
+	    marker->data.count <= ctx->size) {
+		index = marker->data.count - 1;
+	} else if (marker->data.packet_id == 's' &&
+		  marker->data.count*2 <= ctx->size){
+		index = ctx->size / 2 + marker->data.count - 1;
+	}
+
+	if (index >= 0) {
+		ctx->msg.samples[index].id = marker->data.count;
+		ctx->msg.samples[index].ts = ts;
+		ctx->msg.count++;
+	}
+}
+
+static int ptp_marker_find(struct tep_event *event, struct tep_record *record,
+			   int cpu, void *context)
+{
+	struct ptp_markers_context *ctx;
+	struct tep_format_field *field;
+	struct tep_format_field *id;
+	struct ptp_marker *marker;
+
+	ctx = (struct ptp_markers_context *)context;
+
+	/* Make sure this is our event */
+	if (event->id != ctx->ptp->raw_id)
+		return 0;
+	id = tep_find_field(event, "id");
+	field = tep_find_field(event, "buf");
+	if (field && id &&
+	    record->size >= (id->offset + sizeof(struct ptp_marker))) {
+		marker = (struct ptp_marker *)(record->data + id->offset);
+		if (marker->data.local_cid == ctx->clock->local_cid &&
+		    marker->data.remote_cid == ctx->clock->remote_cid &&
+		    marker->series_id == ctx->ptp->series_id &&
+		    marker->data.count)
+			ptp_probe_store(ctx, marker, record->ts);
+	}
+
+	return 0;
+}
+
+static int ptp_calc_offset_fastest(struct clock_sync_context *clock,
+			   struct ptp_clock_result_msg *server,
+			   struct ptp_clock_result_msg *client,
+			   long long *offset_ret, long long *ts_ret,
+			   int *bad_probes)
+{
+	struct ptp_clock_sample *sample_send;
+	long long delta_min = 0;
+	long long offset = 0;
+	long long delta = 0;
+	long long ts = 0;
+	int i;
+
+	*bad_probes = 0;
+	sample_send = server->samples + (server->count / 2);
+	for (i = 0; i * 2 < server->count && i < client->count; i++) {
+		if (!server->samples[i].ts ||
+		    !sample_send[i].ts ||
+		    !client->samples[i].ts ||
+		    server->samples[i].id != sample_send[i].id ||
+		    server->samples[i].id != client->samples[i].id) {
+			(*bad_probes)++;
+			continue;
+		}
+
+		ts = (sample_send[i].ts + server->samples[i].ts) / 2;
+		offset = client->samples[i].ts - ts;
+
+		delta = server->samples[i].ts - sample_send[i].ts;
+		if (!delta_min || delta_min > delta) {
+			delta_min = delta;
+			*offset_ret = offset;
+			*ts_ret = ts;
+		}
+#ifdef TSYNC_DEBUG
+		{
+			struct ptp_clock_sync *ptp;
+
+			ptp = (struct ptp_clock_sync *)clock->proto_data;
+			if (ptp && ptp->debug_fd > 0) {
+				char buff[256];
+
+				sprintf(buff, "%lld %lld %lld\n",
+					ts, client->samples[i].ts, offset);
+				write(ptp->debug_fd, buff, strlen(buff));
+			}
+		}
+#endif
+	}
+
+	return 0;
+}
+
+static int ptp_calc_offset_hist(struct clock_sync_context *clock,
+			   struct ptp_clock_result_msg *server,
+			   struct ptp_clock_result_msg *client,
+			   long long *offset_ret, long long *ts_ret,
+			   int *bad_probes)
+{
+	struct ptp_clock_sample *sample_send;
+	long long timestamps[PTP_SYNC_LOOP];
+	long long offsets[PTP_SYNC_LOOP];
+	int hist[PTP_SYNC_LOOP];
+	long long offset_max = 0;
+	long long offset_min = 0;
+	int ind, max = 0;
+	long long bin;
+	int i, k = 0;
+
+	*bad_probes = 0;
+	memset(hist, 0, sizeof(int) * PTP_SYNC_LOOP);
+	sample_send = server->samples + (server->count / 2);
+	for (i = 0; i * 2 < server->count && i < client->count; i++) {
+		if (!server->samples[i].ts ||
+		    !sample_send[i].ts ||
+		    !client->samples[i].ts ||
+		    server->samples[i].id != sample_send[i].id ||
+		    server->samples[i].id != client->samples[i].id) {
+			(*bad_probes)++;
+			continue;
+		}
+
+		timestamps[k] = (sample_send[i].ts + server->samples[i].ts) / 2;
+		offsets[k] = client->samples[i].ts - timestamps[k];
+		if (!offset_max || offset_max < llabs(offsets[k]))
+			offset_max = llabs(offsets[k]);
+		if (!offset_min || offset_min > llabs(offsets[k]))
+			offset_min = llabs(offsets[k]);
+#ifdef TSYNC_DEBUG
+		{
+			struct ptp_clock_sync *ptp;
+
+			ptp = (struct ptp_clock_sync *)clock->proto_data;
+
+			if (ptp && ptp->debug_fd > 0) {
+				char buff[256];
+
+				sprintf(buff, "%lld %lld %lld\n",
+					timestamps[k],
+					client->samples[i].ts, offsets[k]);
+				write(ptp->debug_fd, buff, strlen(buff));
+			}
+		}
+#endif
+		k++;
+	}
+
+	bin = (offset_max - offset_min) / PTP_SYNC_LOOP;
+	for (i = 0; i < k; i++) {
+		ind = (llabs(offsets[i]) - offset_min) / bin;
+		if (ind < PTP_SYNC_LOOP) {
+			hist[ind]++;
+			if (max < hist[ind]) {
+				max = hist[ind];
+				*offset_ret = offsets[i];
+				*ts_ret = timestamps[i];
+			}
+		}
+	}
+
+	return 0;
+}
+
+static void ntoh_ptp_results(struct ptp_clock_result_msg *msg)
+{
+	int i;
+
+	msg->count = ntohl(msg->count);
+	for (i = 0; i < msg->count; i++) {
+		msg->samples[i].id = ntohl(msg->samples[i].id);
+		msg->samples[i].ts = ntohll(msg->samples[i].ts);
+	}
+	msg->series_id = ntohl(msg->series_id);
+}
+
+
+static void hton_ptp_results(struct ptp_clock_result_msg *msg)
+{
+	int i;
+
+	for (i = 0; i < msg->count; i++) {
+		msg->samples[i].id = htonl(msg->samples[i].id);
+		msg->samples[i].ts = htonll(msg->samples[i].ts);
+	}
+	msg->series_id = htonl(msg->series_id);
+	msg->count = htonl(msg->count);
+}
+
+static inline void ptp_track_clock(struct ptp_markers_context *ctx,
+				   struct ptp_marker *marker)
+{
+	if (ctx->ptp->flags & PTP_FLAG_USE_MARKER) {
+		write(ctx->ptp->marker_fd, marker, sizeof(struct ptp_marker));
+	} else {
+		struct timespec clock;
+		unsigned long long ts;
+
+		clock_gettime(CLOCK_MONOTONIC_RAW, &clock);
+		ts = clock.tv_sec * 1000000000LL;
+		ts += clock.tv_nsec;
+		ptp_probe_store(ctx, marker, ts);
+	}
+}
+
+static int ptp_clock_client(struct tracecmd_time_sync *tsync,
+			    long long *offset, long long *timestamp)
+{
+	struct clock_sync_context *clock_context;
+	struct ptp_clock_offset_msg res_offset;
+	unsigned int sync_proto, sync_msg;
+	struct ptp_clock_start_msg start;
+	struct ptp_markers_context ctx;
+	struct ptp_clock_sync *ptp;
+	struct ptp_marker marker;
+	unsigned int size;
+	char *msg;
+	int count;
+	int ret;
+
+	if (!tsync || !tsync->context || !tsync->msg_handle)
+		return -1;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+	if (clock_context->proto_data == NULL)
+		return -1;
+
+	ptp = (struct ptp_clock_sync *)clock_context->proto_data;
+	size = sizeof(start);
+	msg = (char *)&start;
+	ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+					  &sync_proto, &sync_msg,
+					  &size, &msg);
+	if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+	    sync_msg != PTP_SYNC_PKT_START)
+		return -1;
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					  TRACECMD_TIME_SYNC_PROTO_PTP,
+					  PTP_SYNC_PKT_START, sizeof(start),
+					  (char *)&start);
+	marker.data.local_cid = clock_context->local_cid;
+	marker.data.remote_cid = clock_context->remote_cid;
+	marker.series_id = ntohl(start.series_id);
+	marker.data.packet_id = 'r';
+	ptp->series_id = marker.series_id;
+	ptp->flags = ntohl(start.flags);
+	msg = (char *)&count;
+	size = sizeof(count);
+	ctx.msg.count = 0;
+	ctx.size = PTP_SYNC_LOOP;
+	ctx.ptp = ptp;
+	ctx.clock = clock_context;
+	ctx.msg.series_id = ptp->series_id;
+	while (true) {
+		count = 0;
+		ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+						  &sync_proto, &sync_msg,
+						  &size, &msg);
+		if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+		    sync_msg != PTP_SYNC_PKT_PROBE || !ntohl(count))
+			break;
+		marker.data.count = ntohl(count);
+		ptp_track_clock(&ctx, &marker);
+		ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+						 TRACECMD_TIME_SYNC_PROTO_PTP,
+						 PTP_SYNC_PKT_PROBE,
+						 sizeof(count), (char *)&count);
+		if (ret)
+			break;
+	}
+
+	if (sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+	    sync_msg != PTP_SYNC_PKT_END)
+		return -1;
+
+	if (ptp->flags & PTP_FLAG_USE_MARKER)
+		tracefs_iterate_raw_events(ptp->tep, clock_context->instance,
+					   ptp_marker_find, &ctx);
+
+	hton_ptp_results(&ctx.msg);
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					  TRACECMD_TIME_SYNC_PROTO_PTP,
+					  PTP_SYNC_PKT_PROBES,
+					  sizeof(ctx.msg), (char *)&ctx.msg);
+
+	msg = (char *)&res_offset;
+	size = sizeof(res_offset);
+	ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+					  &sync_proto, &sync_msg,
+					  &size, (char **)&msg);
+	if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+	    sync_msg != PTP_SYNC_PKT_OFFSET)
+		return -1;
+
+	*offset = ntohll(res_offset.offset);
+	*timestamp = ntohll(res_offset.ts);
+
+	return 0;
+}
+
+
+static int ptp_clock_server(struct tracecmd_time_sync *tsync,
+			    long long *offset, long long *timestamp)
+{
+	struct ptp_clock_result_msg *results = NULL;
+	struct clock_sync_context *clock_context;
+	struct ptp_clock_offset_msg res_offset;
+	unsigned int sync_proto, sync_msg;
+	struct ptp_clock_start_msg start;
+	struct ptp_markers_context ctx;
+	int sync_loop = PTP_SYNC_LOOP;
+	struct ptp_clock_sync *ptp;
+	struct ptp_marker marker;
+	unsigned int size;
+	int bad_probes;
+	int count = 1;
+	int msg_count;
+	int msg_ret;
+	char *msg;
+	int ret;
+
+	if (!tsync || !tsync->context || !tsync->msg_handle)
+		return -1;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+	if (clock_context->proto_data == NULL)
+		return -1;
+
+	ptp = (struct ptp_clock_sync *)clock_context->proto_data;
+	ptp->flags = ptp_flags;
+	memset(&start, 0, sizeof(start));
+	start.series_id = htonl(ptp->series_id + 1);
+	start.flags = htonl(ptp->flags);
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					 TRACECMD_TIME_SYNC_PROTO_PTP,
+					 PTP_SYNC_PKT_START, sizeof(start),
+					 (char *)&start);
+	if (!ret)
+		ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+						  &sync_proto, &sync_msg,
+						  NULL, NULL);
+	if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+	    sync_msg != PTP_SYNC_PKT_START)
+		return -1;
+
+	tracefs_instance_file_write(clock_context->instance, "trace", "\0");
+
+	ptp->series_id++;
+	marker.data.local_cid = clock_context->local_cid;
+	marker.data.remote_cid = clock_context->remote_cid;
+	marker.series_id = ptp->series_id;
+	msg = (char *)&msg_ret;
+	size = sizeof(msg_ret);
+	ctx.size = 2*PTP_SYNC_LOOP;
+	ctx.ptp = ptp;
+	ctx.clock = clock_context;
+	ctx.msg.count = 0;
+	ctx.msg.series_id = ptp->series_id;
+	do {
+		marker.data.count = count++;
+		marker.data.packet_id = 's';
+		msg_count = htonl(marker.data.count);
+		ptp_track_clock(&ctx, &marker);
+		ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+						 TRACECMD_TIME_SYNC_PROTO_PTP,
+						 PTP_SYNC_PKT_PROBE,
+						 sizeof(msg_count),
+						 (char *)&msg_count);
+		if (!ret)
+			ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+							  &sync_proto, &sync_msg,
+							  &size, &msg);
+
+		marker.data.packet_id = 'r';
+		ptp_track_clock(&ctx, &marker);
+		if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+		    sync_msg != PTP_SYNC_PKT_PROBE ||
+		    ntohl(msg_ret) != marker.data.count)
+			break;
+	} while (--sync_loop);
+
+	if (sync_loop)
+		return -1;
+
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					  TRACECMD_TIME_SYNC_PROTO_PTP,
+					  PTP_SYNC_PKT_END, 0, NULL);
+
+	size = 0;
+	ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+					  &sync_proto, &sync_msg,
+					  &size, (char **)&results);
+	if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_PTP ||
+	    sync_msg != PTP_SYNC_PKT_PROBES || size == 0 || results == NULL)
+		return -1;
+
+	ntoh_ptp_results(results);
+	if (ptp->flags & PTP_FLAG_USE_MARKER)
+		tracefs_iterate_raw_events(ptp->tep, clock_context->instance,
+					   ptp_marker_find, &ctx);
+	if (ptp->flags & PTP_FLAG_FASTEST_RESPONSE)
+		ptp_calc_offset_fastest(clock_context, &ctx.msg, results, offset,
+					timestamp, &bad_probes);
+	else
+		ptp_calc_offset_hist(clock_context, &ctx.msg, results, offset,
+				     timestamp, &bad_probes);
+#ifdef TSYNC_DEBUG
+	{
+		char buff[256];
+		int res_fd;
+
+		sprintf(buff, "res-cid%d.txt", clock_context->remote_cid);
+
+		res_fd = open(buff, O_WRONLY|O_APPEND, 0644);
+		if (res_fd > 0) {
+			if (*offset && *timestamp) {
+				sprintf(buff, "%d %lld %lld\n",
+					ptp->series_id, *offset, *timestamp);
+				write(res_fd, buff, strlen(buff));
+			}
+			close(res_fd);
+		}
+
+		printf("\n calculated offset %d: %lld, %d probes, filtered out %d, PTP flags 0x%X\n\r",
+			ptp->series_id, *offset, results->count, bad_probes, ptp->flags);
+		if (ptp && ptp->debug_fd > 0) {
+			sprintf(buff, "%lld %lld 0\n", *offset, *timestamp);
+			write(ptp->debug_fd, buff, strlen(buff));
+			close(ptp->debug_fd);
+			ptp->debug_fd = -1;
+		}
+
+	}
+#endif
+
+	res_offset.offset = htonll(*offset);
+	res_offset.ts = htonll(*timestamp);
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					  TRACECMD_TIME_SYNC_PROTO_PTP,
+					  PTP_SYNC_PKT_OFFSET,
+					  sizeof(res_offset),
+					  (char *)&res_offset);
+
+	free(results);
+	return 0;
+}
+
+static int ptp_clock_sync_calc(struct tracecmd_time_sync *tsync,
+			       long long *offset, long long *timestamp)
+{
+	struct clock_sync_context *clock_context;
+	int ret;
+
+	if (!tsync || !tsync->context)
+		return -1;
+	clock_context = (struct clock_sync_context *)tsync->context;
+
+#ifdef TSYNC_DEBUG
+	if (clock_context->is_server) {
+		struct ptp_clock_sync *ptp;
+		char buff[256];
+
+		ptp = (struct ptp_clock_sync *)clock_context->proto_data;
+		if (ptp->debug_fd > 0)
+			close(ptp->debug_fd);
+		sprintf(buff, "s-cid%d_%d.txt",
+				clock_context->remote_cid, ptp->series_id+1);
+		ptp->debug_fd = open(buff, O_CREAT|O_WRONLY|O_TRUNC, 0644);
+	}
+#endif
+
+
+	if (clock_context->is_server)
+		ret = ptp_clock_server(tsync, offset, timestamp);
+	else
+		ret = ptp_clock_client(tsync, offset, timestamp);
+
+	return ret;
+}
+
+int ptp_clock_sync_register(void)
+{
+	return tracecmd_tsync_proto_register(TRACECMD_TIME_SYNC_PROTO_PTP,
+					     TRACECMD_TIME_SYNC_PROTO_PTP_WEIGHT,
+					     ptp_clock_sync_init,
+					     ptp_clock_sync_free,
+					     ptp_clock_sync_calc);
+
+}
+
+int ptp_clock_sync_unregister(void)
+{
+	return tracecmd_tsync_proto_unregister(TRACECMD_TIME_SYNC_PROTO_PTP);
+}
diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
index 35a41394..e294698b 100644
--- a/lib/trace-cmd/trace-timesync.c
+++ b/lib/trace-cmd/trace-timesync.c
@@ -46,6 +46,14 @@ static struct tsync_proto *tsync_proto_find(unsigned int proto_id)
 	return NULL;
 }
 
+/**
+ * tracecmd_tsync_init - Initialize the global, per task, time sync data.
+ */
+void tracecmd_tsync_init(void)
+{
+	ptp_clock_sync_register();
+}
+
 int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
 				int (*init)(struct tracecmd_time_sync *),
 				int (*free)(struct tracecmd_time_sync *),
diff --git a/tracecmd/trace-agent.c b/tracecmd/trace-agent.c
index b5816966..2fd09dc3 100644
--- a/tracecmd/trace-agent.c
+++ b/tracecmd/trace-agent.c
@@ -247,6 +247,8 @@ static void agent_serve(unsigned int port)
 	if (sd < 0)
 		die("Failed to open vsocket");
 
+	tracecmd_tsync_init();
+
 	if (!get_local_cid(&cid))
 		printf("listening on @%u:%u\n", cid, port);
 
diff --git a/tracecmd/trace-record.c b/tracecmd/trace-record.c
index bd004574..f72dfaaf 100644
--- a/tracecmd/trace-record.c
+++ b/tracecmd/trace-record.c
@@ -6425,10 +6425,6 @@ static bool has_local_instances(void)
 	return false;
 }
 
-/*
- * This function contains common code for the following commands:
- * record, start, stream, profile.
- */
 static void record_trace(int argc, char **argv,
 			 struct common_record_context *ctx)
 {
@@ -6586,12 +6582,23 @@ static void record_trace(int argc, char **argv,
 	finalize_record_trace(ctx);
 }
 
+/*
+ * This function contains common code for the following commands:
+ * record, start, stream, profile.
+ */
+static void record_trace_command(int argc, char **argv,
+				 struct common_record_context *ctx)
+{
+	tracecmd_tsync_init();
+	record_trace(argc, argv, ctx);
+}
+
 void trace_start(int argc, char **argv)
 {
 	struct common_record_context ctx;
 
 	parse_record_options(argc, argv, CMD_start, &ctx);
-	record_trace(argc, argv, &ctx);
+	record_trace_command(argc, argv, &ctx);
 	exit(0);
 }
 
@@ -6680,7 +6687,7 @@ void trace_stream(int argc, char **argv)
 	struct common_record_context ctx;
 
 	parse_record_options(argc, argv, CMD_stream, &ctx);
-	record_trace(argc, argv, &ctx);
+	record_trace_command(argc, argv, &ctx);
 	exit(0);
 }
 
@@ -6699,7 +6706,7 @@ void trace_profile(int argc, char **argv)
 	if (!buffer_instances)
 		top_instance.flags |= BUFFER_FL_PROFILE;
 
-	record_trace(argc, argv, &ctx);
+	record_trace_command(argc, argv, &ctx);
 	do_trace_profile();
 	exit(0);
 }
@@ -6718,7 +6725,7 @@ void trace_record(int argc, char **argv)
 	struct common_record_context ctx;
 
 	parse_record_options(argc, argv, CMD_record, &ctx);
-	record_trace(argc, argv, &ctx);
+	record_trace_command(argc, argv, &ctx);
 	exit(0);
 }
 
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 02/10] trace-cmd: Debug scripts for PTP-like algorithm for host - guest timestamp synchronization
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name Tzvetomir Stoyanov (VMware)
                   ` (7 subsequent siblings)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

These scripts can be used to visualise debug files, written when the PTP-like algorithm
is compiled with TSYNC_DEBUG defined. The files are located in the guest machine:
    s-cid*.txt - For each offset calculation: host and guest clocks and calculated offset.
    res-cid*.txt - For each tracing session: all calculated clock offsets.

tsync_hist.py plots a histogram, using data from a s-cid*.txt file:
	"python tsync_hist.py s-cid2_1.txt"
tsync_res.py plots a line, using data from res-cid*.txt file:
	"python tsync_res.py res-cid2.txt"

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 scripts/debug/tsync_hist.py | 57 +++++++++++++++++++++++++++++++++++++
 scripts/debug/tsync_readme  | 12 ++++++++
 scripts/debug/tsync_res.py  | 46 ++++++++++++++++++++++++++++++
 3 files changed, 115 insertions(+)
 create mode 100644 scripts/debug/tsync_hist.py
 create mode 100644 scripts/debug/tsync_readme
 create mode 100644 scripts/debug/tsync_res.py

diff --git a/scripts/debug/tsync_hist.py b/scripts/debug/tsync_hist.py
new file mode 100644
index 00000000..819d1e8f
--- /dev/null
+++ b/scripts/debug/tsync_hist.py
@@ -0,0 +1,57 @@
+# SPDX-License-Identifier: GPL-2.0
+#
+# Copyright (C) 2019, VMware Inc, Tzvetomir Stoyanov <tz.stoyanov@gmail.com>
+# Copyright (C) 2019, VMware Inc, Yordan Karadzhov <ykaradzhov@vmware.com>
+
+
+import matplotlib.pyplot as plt
+import matplotlib.lines as mlines
+import numpy as np
+import sys
+
+def newline(p1, p2):
+    ax = plt.gca()
+    xmin, xmax = ax.get_xbound()
+
+    if(p2[0] == p1[0]):
+        xmin = xmax = p1[0]
+        ymin, ymax = ax.get_ybound()
+    else:
+        ymax = p1[1]+(p2[1]-p1[1])/(p2[0]-p1[0])*(xmax-p1[0])
+        ymin = p1[1]+(p2[1]-p1[1])/(p2[0]-p1[0])*(xmin-p1[0])
+
+    l = mlines.Line2D([xmin,xmax], [ymin,ymax], color='red')
+    ax.add_line(l)
+    return l
+
+
+data = np.loadtxt(fname = sys.argv[1])
+selected_ts  = data[-1, 1]
+selected_ofs = data[-1, 0]
+data = data[:-1,:]
+
+x = data[:, 1] - data[:, 0]
+
+mean = x.mean()
+std = x.std()
+
+num_bins = 500
+min = x.min() #+ .4 * (x.max() - x.min())
+max = x.max() #- .4 * (x.max() - x.min())
+bins = np.linspace(min, max, num_bins, endpoint = False, dtype=int)
+
+fig, ax = plt.subplots()
+
+# the histogram of the data
+n, bins, patches = ax.hist(x, bins, histtype=u'step');
+
+ax.set_xlabel('clock offset [$\mu$s]')
+ax.set_ylabel('entries')
+ax.set_title("$\sigma$=%i" % std)
+
+x1, y1 = [selected_ofs, min], [selected_ofs, max]
+newline(x1, y1)
+
+# Tweak spacing to prevent clipping of ylabel
+fig.tight_layout()
+plt.show()
diff --git a/scripts/debug/tsync_readme b/scripts/debug/tsync_readme
new file mode 100644
index 00000000..f3ebb25d
--- /dev/null
+++ b/scripts/debug/tsync_readme
@@ -0,0 +1,12 @@
+PTP-like algorithm debug
+========================
+
+tsync_*.py scripts can be used to visualise debug files, written when the PTP-like algorithm
+is compiled with TSYNC_DEBUG defined. The files are located in the guest machine:
+    s-cid*.txt - For each offset calculation: host and guest clocks and calculated offset.
+    res-cid*.txt - For each tracing session: all calculated clock offsets.
+
+tsync_hist.py plots a histogram, using data from a s-cid*.txt file:
+	"python tsync_hist.py s-cid2_1.txt"
+tsync_res.py plots a line, using data from res-cid*.txt file:
+	"python tsync_res.py res-cid2.txt"
diff --git a/scripts/debug/tsync_res.py b/scripts/debug/tsync_res.py
new file mode 100644
index 00000000..7d109863
--- /dev/null
+++ b/scripts/debug/tsync_res.py
@@ -0,0 +1,46 @@
+# SPDX-License-Identifier: GPL-2.0
+#
+# Copyright (C) 2019, VMware Inc, Tzvetomir Stoyanov <tz.stoyanov@gmail.com>
+# Copyright (C) 2019, VMware Inc, Yordan Karadzhov <ykaradzhov@vmware.com>
+
+
+import matplotlib.pyplot as plt
+import matplotlib.lines as mlines
+import numpy as np
+import sys
+
+def newline(p1, p2):
+    ax = plt.gca()
+    xmin, xmax = ax.get_xbound()
+
+    if(p2[0] == p1[0]):
+        xmin = xmax = p1[0]
+        ymin, ymax = ax.get_ybound()
+    else:
+        ymax = p1[1]+(p2[1]-p1[1])/(p2[0]-p1[0])*(xmax-p1[0])
+        ymin = p1[1]+(p2[1]-p1[1])/(p2[0]-p1[0])*(xmin-p1[0])
+
+    l = mlines.Line2D([xmin,xmax], [ymin,ymax], color='red')
+    ax.add_line(l)
+    return l
+
+data = np.loadtxt(fname = sys.argv[1])
+x = data[:, 0]
+y = data[:, 1]
+
+fig, ax = plt.subplots()
+
+ax.set_xlabel('samples (t)')
+ax.set_ylabel('clock offset')
+ax.set_title("$\delta$=%i ns" % (max(y) - min(y)))
+
+l = mlines.Line2D(x, y)
+ax.add_line(l)
+ax.set_xlim(min(x), max(x))
+ax.set_ylim(min(y), max(y) )
+
+print(min(y), max(y), max(y) - min(y))
+
+# Tweak spacing to prevent clipping of ylabel
+fig.tight_layout()
+plt.show()
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 02/10] trace-cmd: Debug scripts for " Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-22  1:26   ` Steven Rostedt
  2020-10-09 14:03 ` [PATCH v24 04/10] trace-cmd: Move VM related logic in a separate file Tzvetomir Stoyanov (VMware)
                   ` (6 subsequent siblings)
  9 siblings, 1 reply; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

Added enum with ftrace clock IDs and APIs to convert ftrace name to ID
and vice versa, as part of libtracecmd.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 include/trace-cmd/trace-cmd.h | 16 +++++++++++++
 lib/trace-cmd/trace-util.c    | 45 +++++++++++++++++++++++++++++++++++
 2 files changed, 61 insertions(+)

diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
index f9c1f843..393a2e7b 100644
--- a/include/trace-cmd/trace-cmd.h
+++ b/include/trace-cmd/trace-cmd.h
@@ -415,6 +415,22 @@ int tracecmd_msg_recv_time_sync(struct tracecmd_msg_handle *msg_handle,
 				unsigned int *sync_msg_id,
 				unsigned int *payload_size, char **payload);
 
+enum tracecmd_clocks {
+	TRACECMD_CLOCK_UNKNOWN	= 0,
+	TRACECMD_CLOCK_LOCAL	= 1,
+	TRACECMD_CLOCK_GLOBAL	= 1 << 1,
+	TRACECMD_CLOCK_COUNTER	= 1 << 2,
+	TRACECMD_CLOCK_UPTIME	= 1 << 3,
+	TRACECMD_CLOCK_PERF	= 1 << 4,
+	TRACECMD_CLOCK_MONO	= 1 << 5,
+	TRACECMD_CLOCK_MONO_RAW	= 1 << 6,
+	TRACECMD_CLOCK_BOOT	= 1 << 7,
+	TRACECMD_CLOCK_X86_TSC	= 1 << 8
+};
+
+enum tracecmd_clocks tracecmd_clock_str2id(const char *clock);
+char *tracecmd_clock_id2str(enum tracecmd_clocks clock);
+
 /* --- Timestamp synchronization --- */
 
 enum{
diff --git a/lib/trace-cmd/trace-util.c b/lib/trace-cmd/trace-util.c
index 0ead96ea..e20362e3 100644
--- a/lib/trace-cmd/trace-util.c
+++ b/lib/trace-cmd/trace-util.c
@@ -33,6 +33,51 @@ static bool debug;
 
 static FILE *logfp;
 
+const static struct {
+	char *clock_str;
+	enum tracecmd_clocks clock_id;
+} trace_clocks[] = {
+	{"local", TRACECMD_CLOCK_LOCAL},
+	{"global", TRACECMD_CLOCK_GLOBAL},
+	{"counter", TRACECMD_CLOCK_COUNTER},
+	{"uptime", TRACECMD_CLOCK_UPTIME},
+	{"perf", TRACECMD_CLOCK_PERF},
+	{"mono", TRACECMD_CLOCK_MONO},
+	{"mono_raw", TRACECMD_CLOCK_MONO_RAW},
+	{"boot", TRACECMD_CLOCK_BOOT},
+	{"x86-tsc", TRACECMD_CLOCK_X86_TSC},
+	{NULL, -1}
+};
+
+/**
+ * tracecmd_clock_str2id - Convert ftrace clock name to clock ID
+ * @clock: Ftrace clock name
+ * Returns ID of the ftrace clock
+ */
+enum tracecmd_clocks tracecmd_clock_str2id(const char *clock)
+{
+	for (int i = 0; trace_clocks[i].clock_str; i++) {
+		if (!strncmp(clock, trace_clocks[i].clock_str,
+		    strlen(trace_clocks[i].clock_str)))
+			return trace_clocks[i].clock_id;
+	}
+	return TRACECMD_CLOCK_UNKNOWN;
+}
+
+/**
+ * tracecmd_clock_id2str - Convert clock ID to ftare clock name
+ * @clock: Clock ID
+ * Returns name of a ftrace clock
+ */
+char *tracecmd_clock_id2str(enum tracecmd_clocks clock)
+{
+	for (int i = 0; trace_clocks[i].clock_str; i++) {
+		if (trace_clocks[i].clock_id == clock)
+			return trace_clocks[i].clock_str;
+	}
+	return NULL;
+}
+
 /**
  * tracecmd_set_debug - Set debug mode of the tracecmd library
  * @set_debug: The new "debug" mode. If true, the tracecmd library is
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 04/10] trace-cmd: Move VM related logic in a separate file
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (2 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 05/10] trace-cmd: Add clock parameter to timestamp synchronization plugins Tzvetomir Stoyanov (VMware)
                   ` (5 subsequent siblings)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

All trace-cmd internal functions related to VM / guest resolving and
mappings are moved from trace-record.c to trace-vm.c. Internal APIs are
added to access the guest database, so this functionality can be used by
other logic, internally in the trace-cmd context.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 tracecmd/Makefile              |   1 +
 tracecmd/include/trace-local.h |  20 +++
 tracecmd/trace-record.c        | 232 +++------------------------------
 tracecmd/trace-vm.c            | 214 ++++++++++++++++++++++++++++++
 4 files changed, 250 insertions(+), 217 deletions(-)
 create mode 100644 tracecmd/trace-vm.c

diff --git a/tracecmd/Makefile b/tracecmd/Makefile
index 5e59adf8..09f13cf9 100644
--- a/tracecmd/Makefile
+++ b/tracecmd/Makefile
@@ -31,6 +31,7 @@ TRACE_CMD_OBJS += trace-show.o
 TRACE_CMD_OBJS += trace-list.o
 TRACE_CMD_OBJS += trace-usage.o
 TRACE_CMD_OBJS += trace-dump.o
+TRACE_CMD_OBJS += trace-vm.o
 ifeq ($(VSOCK_DEFINED), 1)
 TRACE_CMD_OBJS += trace-tsync.o
 endif
diff --git a/tracecmd/include/trace-local.h b/tracecmd/include/trace-local.h
index d148aa16..49549f87 100644
--- a/tracecmd/include/trace-local.h
+++ b/tracecmd/include/trace-local.h
@@ -8,6 +8,7 @@
 
 #include <sys/types.h>
 #include <dirent.h>	/* for DIR */
+#include <ctype.h>	/* for isdigit() */
 
 #include "trace-cmd.h"
 #include "event-utils.h"
@@ -283,6 +284,17 @@ void update_first_instance(struct buffer_instance *instance, int topt);
 
 void show_instance_file(struct buffer_instance *instance, const char *name);
 
+struct trace_guest {
+	char *name;
+	int cid;
+	int pid;
+	int cpu_max;
+	int *cpu_pid;
+};
+struct trace_guest *get_guest_by_cid(unsigned int guest_cid);
+struct trace_guest *get_guest_by_name(char *name);
+void read_qemu_guests(void);
+int get_guest_pid(unsigned int guest_cid);
 int get_guest_vcpu_pid(unsigned int guest_cid, unsigned int guest_vcpu);
 
 /* moved from trace-cmd.h */
@@ -314,4 +326,12 @@ void *malloc_or_die(unsigned int size); /* Can be overridden */
 void __noreturn __die(const char *fmt, ...);
 void __noreturn _vdie(const char *fmt, va_list ap);
 
+static inline bool is_digits(const char *s)
+{
+	for (; *s; s++)
+		if (!isdigit(*s))
+			return false;
+	return true;
+}
+
 #endif /* __TRACE_LOCAL_H */
diff --git a/tracecmd/trace-record.c b/tracecmd/trace-record.c
index f72dfaaf..9149cf76 100644
--- a/tracecmd/trace-record.c
+++ b/tracecmd/trace-record.c
@@ -3190,239 +3190,37 @@ static int do_accept(int sd)
 	return -1;
 }
 
-static bool is_digits(const char *s)
+static char *parse_guest_name(char *gname, int *cid, int *port)
 {
-	for (; *s; s++)
-		if (!isdigit(*s))
-			return false;
-	return true;
-}
-
-struct guest {
-	char *name;
-	int cid;
-	int pid;
-	int cpu_max;
-	int *cpu_pid;
-};
-
-static struct guest *guests;
-static size_t guests_len;
-
-static int set_vcpu_pid_mapping(struct guest *guest, int cpu, int pid)
-{
-	int *cpu_pid;
-	int i;
-
-	if (cpu >= guest->cpu_max) {
-		cpu_pid = realloc(guest->cpu_pid, (cpu + 1) * sizeof(int));
-		if (!cpu_pid)
-			return -1;
-		/* Handle sparse CPU numbers */
-		for (i = guest->cpu_max; i < cpu; i++)
-			cpu_pid[i] = -1;
-		guest->cpu_max = cpu + 1;
-		guest->cpu_pid = cpu_pid;
-	}
-	guest->cpu_pid[cpu] = pid;
-	return 0;
-}
-
-static struct guest *get_guest_info(unsigned int guest_cid)
-{
-	int i;
-
-	if (!guests)
-		return NULL;
-
-	for (i = 0; i < guests_len; i++)
-		if (guest_cid == guests[i].cid)
-			return guests + i;
-	return NULL;
-}
-
-static char *get_qemu_guest_name(char *arg)
-{
-	char *tok, *end = arg;
-
-	while ((tok = strsep(&end, ","))) {
-		if (strncmp(tok, "guest=", 6) == 0)
-			return tok + 6;
-	}
-
-	return arg;
-}
-
-static int read_qemu_guests_pids(char *guest_task, struct guest *guest)
-{
-	struct dirent *entry;
-	char path[PATH_MAX];
-	char *buf = NULL;
-	size_t n = 0;
-	int ret = 0;
-	long vcpu;
-	long pid;
-	DIR *dir;
-	FILE *f;
-
-	snprintf(path, sizeof(path), "/proc/%s/task", guest_task);
-	dir = opendir(path);
-	if (!dir)
-		return -1;
-
-	while (!ret && (entry = readdir(dir))) {
-		if (!(entry->d_type == DT_DIR && is_digits(entry->d_name)))
-			continue;
-
-		snprintf(path, sizeof(path), "/proc/%s/task/%s/comm",
-			 guest_task, entry->d_name);
-		f = fopen(path, "r");
-		if (!f)
-			continue;
-
-		if (getline(&buf, &n, f) >= 0 &&
-		    strncmp(buf, "CPU ", 4) == 0) {
-			vcpu = strtol(buf + 4, NULL, 10);
-			pid = strtol(entry->d_name, NULL, 10);
-			if (vcpu < INT_MAX && pid < INT_MAX &&
-			    vcpu >= 0 && pid >= 0) {
-				if (set_vcpu_pid_mapping(guest, vcpu, pid))
-					ret = -1;
-			}
-		}
-
-		fclose(f);
-	}
-	free(buf);
-	return ret;
-}
-
-static void read_qemu_guests(void)
-{
-	static bool initialized;
-	struct dirent *entry;
-	char path[PATH_MAX];
-	DIR *dir;
-
-	if (initialized)
-		return;
-
-	initialized = true;
-	dir = opendir("/proc");
-	if (!dir)
-		die("Can not open /proc");
-
-	while ((entry = readdir(dir))) {
-		bool is_qemu = false, last_was_name = false;
-		struct guest guest = {};
-		char *p, *arg = NULL;
-		size_t arg_size = 0;
-		FILE *f;
-
-		if (!(entry->d_type == DT_DIR && is_digits(entry->d_name)))
-			continue;
-
-		guest.pid = atoi(entry->d_name);
-		snprintf(path, sizeof(path), "/proc/%s/cmdline", entry->d_name);
-		f = fopen(path, "r");
-		if (!f)
-			continue;
-
-		while (getdelim(&arg, &arg_size, 0, f) != -1) {
-			if (!is_qemu && strstr(arg, "qemu-system-")) {
-				is_qemu = true;
-				continue;
-			}
-
-			if (!is_qemu)
-				continue;
-
-			if (strcmp(arg, "-name") == 0) {
-				last_was_name = true;
-				continue;
-			}
-
-			if (last_was_name) {
-				guest.name = strdup(get_qemu_guest_name(arg));
-				if (!guest.name)
-					die("allocating guest name");
-				last_was_name = false;
-				continue;
-			}
-
-			p = strstr(arg, "guest-cid=");
-			if (p) {
-				guest.cid = atoi(p + 10);
-				continue;
-			}
-		}
-
-		if (!is_qemu)
-			goto next;
-
-		if (read_qemu_guests_pids(entry->d_name, &guest))
-			warning("Failed to retrieve VPCU - PID mapping for guest %s",
-					guest.name ? guest.name : "Unknown");
-
-		guests = realloc(guests, (guests_len + 1) * sizeof(*guests));
-		if (!guests)
-			die("Can not allocate guest buffer");
-		guests[guests_len++] = guest;
-
-next:
-		free(arg);
-		fclose(f);
-	}
-
-	closedir(dir);
-}
-
-static char *parse_guest_name(char *guest, int *cid, int *port)
-{
-	size_t i;
+	struct trace_guest *guest;
 	char *p;
 
 	*port = -1;
-	p = strrchr(guest, ':');
+	p = strrchr(gname, ':');
 	if (p) {
 		*p = '\0';
 		*port = atoi(p + 1);
 	}
 
 	*cid = -1;
-	p = strrchr(guest, '@');
+	p = strrchr(gname, '@');
 	if (p) {
 		*p = '\0';
 		*cid = atoi(p + 1);
-	} else if (is_digits(guest))
-		*cid = atoi(guest);
+	} else if (is_digits(gname))
+		*cid = atoi(gname);
 
 	read_qemu_guests();
-	for (i = 0; i < guests_len; i++) {
-		if ((*cid > 0 && *cid == guests[i].cid) ||
-		    strcmp(guest, guests[i].name) == 0) {
-			*cid = guests[i].cid;
-			return guests[i].name;
-		}
+	if (*cid > 0)
+		guest = get_guest_by_cid(*cid);
+	else
+		guest = get_guest_by_name(gname);
+	if (guest) {
+		*cid = guest->cid;
+		return guest->name;
 	}
 
-	return guest;
-}
-
-int get_guest_vcpu_pid(unsigned int guest_cid, unsigned int guest_vcpu)
-{
-	int i;
-
-	if (!guests)
-		return -1;
-
-	for (i = 0; i < guests_len; i++) {
-		if (guests[i].cpu_pid < 0 || guest_vcpu >= guests[i].cpu_max)
-			continue;
-		if (guest_cid == guests[i].cid)
-			return guests[i].cpu_pid[guest_vcpu];
-	}
-	return -1;
+	return gname;
 }
 
 static void set_prio(int prio)
@@ -4088,7 +3886,7 @@ static void append_buffer(struct tracecmd_output *handle,
 static void
 add_guest_info(struct tracecmd_output *handle, struct buffer_instance *instance)
 {
-	struct guest *guest = get_guest_info(instance->cid);
+	struct trace_guest *guest = get_guest_by_cid(instance->cid);
 	char *buf, *p;
 	int size;
 	int i;
diff --git a/tracecmd/trace-vm.c b/tracecmd/trace-vm.c
new file mode 100644
index 00000000..c8924ece
--- /dev/null
+++ b/tracecmd/trace-vm.c
@@ -0,0 +1,214 @@
+// SPDX-License-Identifier: GPL-2.0
+/*
+ * Copyright (C) 2008, 2009, 2010 Red Hat Inc, Steven Rostedt <srostedt@redhat.com>
+ * Copyright (C) 2020, VMware, Tzvetomir Stoyanov <tz.stoyanov@gmail.com>
+ * ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+ */
+#include <stdio.h>
+#include <stdlib.h>
+#include <sys/types.h>
+#include <dirent.h>
+#include <limits.h>
+
+#include "trace-local.h"
+#include "trace-msg.h"
+
+static struct trace_guest *guests;
+static size_t guests_len;
+
+static int set_vcpu_pid_mapping(struct trace_guest *guest, int cpu, int pid)
+{
+	int *cpu_pid;
+	int i;
+
+	if (cpu >= guest->cpu_max) {
+		cpu_pid = realloc(guest->cpu_pid, (cpu + 1) * sizeof(int));
+		if (!cpu_pid)
+			return -1;
+		/* Handle sparse CPU numbers */
+		for (i = guest->cpu_max; i < cpu; i++)
+			cpu_pid[i] = -1;
+		guest->cpu_max = cpu + 1;
+		guest->cpu_pid = cpu_pid;
+	}
+	guest->cpu_pid[cpu] = pid;
+	return 0;
+}
+
+struct trace_guest *get_guest_by_cid(unsigned int guest_cid)
+{
+	int i;
+
+	if (!guests)
+		return NULL;
+
+	for (i = 0; i < guests_len; i++)
+		if (guest_cid == guests[i].cid)
+			return guests + i;
+	return NULL;
+}
+
+struct trace_guest *get_guest_by_name(char *name)
+{
+	int i;
+
+	if (!guests)
+		return NULL;
+
+	for (i = 0; i < guests_len; i++)
+		if (strcmp(name, guests[i].name) == 0)
+			return guests + i;
+	return NULL;
+}
+
+static char *get_qemu_guest_name(char *arg)
+{
+	char *tok, *end = arg;
+
+	while ((tok = strsep(&end, ","))) {
+		if (strncmp(tok, "guest=", 6) == 0)
+			return tok + 6;
+	}
+
+	return arg;
+}
+
+static int read_qemu_guests_pids(char *guest_task, struct trace_guest *guest)
+{
+	struct dirent *entry;
+	char path[PATH_MAX];
+	char *buf = NULL;
+	size_t n = 0;
+	int ret = 0;
+	long vcpu;
+	long pid;
+	DIR *dir;
+	FILE *f;
+
+	snprintf(path, sizeof(path), "/proc/%s/task", guest_task);
+	dir = opendir(path);
+	if (!dir)
+		return -1;
+
+	while (!ret && (entry = readdir(dir))) {
+		if (!(entry->d_type == DT_DIR && is_digits(entry->d_name)))
+			continue;
+
+		snprintf(path, sizeof(path), "/proc/%s/task/%s/comm",
+			 guest_task, entry->d_name);
+		f = fopen(path, "r");
+		if (!f)
+			continue;
+
+		if (getline(&buf, &n, f) >= 0 &&
+		    strncmp(buf, "CPU ", 4) == 0) {
+			vcpu = strtol(buf + 4, NULL, 10);
+			pid = strtol(entry->d_name, NULL, 10);
+			if (vcpu < INT_MAX && pid < INT_MAX &&
+			    vcpu >= 0 && pid >= 0) {
+				if (set_vcpu_pid_mapping(guest, vcpu, pid))
+					ret = -1;
+			}
+		}
+
+		fclose(f);
+	}
+	free(buf);
+	return ret;
+}
+
+void read_qemu_guests(void)
+{
+	static bool initialized;
+	struct dirent *entry;
+	char path[PATH_MAX];
+	DIR *dir;
+
+	if (initialized)
+		return;
+
+	initialized = true;
+	dir = opendir("/proc");
+	if (!dir)
+		die("Can not open /proc");
+
+	while ((entry = readdir(dir))) {
+		bool is_qemu = false, last_was_name = false;
+		struct trace_guest guest = {};
+		char *p, *arg = NULL;
+		size_t arg_size = 0;
+		FILE *f;
+
+		if (!(entry->d_type == DT_DIR && is_digits(entry->d_name)))
+			continue;
+
+		guest.pid = atoi(entry->d_name);
+		snprintf(path, sizeof(path), "/proc/%s/cmdline", entry->d_name);
+		f = fopen(path, "r");
+		if (!f)
+			continue;
+
+		while (getdelim(&arg, &arg_size, 0, f) != -1) {
+			if (!is_qemu && strstr(arg, "qemu-system-")) {
+				is_qemu = true;
+				continue;
+			}
+
+			if (!is_qemu)
+				continue;
+
+			if (strcmp(arg, "-name") == 0) {
+				last_was_name = true;
+				continue;
+			}
+
+			if (last_was_name) {
+				guest.name = strdup(get_qemu_guest_name(arg));
+				if (!guest.name)
+					die("allocating guest name");
+				last_was_name = false;
+				continue;
+			}
+
+			p = strstr(arg, "guest-cid=");
+			if (p) {
+				guest.cid = atoi(p + 10);
+				continue;
+			}
+		}
+
+		if (!is_qemu)
+			goto next;
+
+		if (read_qemu_guests_pids(entry->d_name, &guest))
+			warning("Failed to retrieve VPCU - PID mapping for guest %s",
+					guest.name ? guest.name : "Unknown");
+
+		guests = realloc(guests, (guests_len + 1) * sizeof(*guests));
+		if (!guests)
+			die("Can not allocate guest buffer");
+		guests[guests_len++] = guest;
+
+next:
+		free(arg);
+		fclose(f);
+	}
+
+	closedir(dir);
+}
+
+int get_guest_vcpu_pid(unsigned int guest_cid, unsigned int guest_vcpu)
+{
+	int i;
+
+	if (!guests)
+		return -1;
+
+	for (i = 0; i < guests_len; i++) {
+		if (guests[i].cpu_pid < 0 || guest_vcpu >= guests[i].cpu_max)
+			continue;
+		if (guest_cid == guests[i].cid)
+			return guests[i].cpu_pid[guest_vcpu];
+	}
+	return -1;
+}
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 05/10] trace-cmd: Add clock parameter to timestamp synchronization plugins
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (3 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 04/10] trace-cmd: Move VM related logic in a separate file Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-22  1:39   ` Steven Rostedt
  2020-10-09 14:03 ` [PATCH v24 06/10] trace-cmd: Add role " Tzvetomir Stoyanov (VMware)
                   ` (4 subsequent siblings)
  9 siblings, 1 reply; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

Some timestamp synchronization plugins may not support all ftrace
clocks. Added logic to timestamp synchronization plugins to declare what
ftace clocks they support. Added logic to select plugin depending on the
ftrace clock used in the current trace session and supported clocks.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 include/trace-cmd/trace-cmd.h             |  4 ++--
 lib/trace-cmd/include/trace-tsync-local.h |  1 +
 lib/trace-cmd/trace-timesync-ptp.c        |  1 +
 lib/trace-cmd/trace-timesync.c            | 21 +++++++++++++++++++--
 tracecmd/trace-record.c                   |  4 +++-
 tracecmd/trace-tsync.c                    |  2 +-
 6 files changed, 27 insertions(+), 6 deletions(-)

diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
index 393a2e7b..66b5d02b 100644
--- a/include/trace-cmd/trace-cmd.h
+++ b/include/trace-cmd/trace-cmd.h
@@ -455,8 +455,8 @@ struct tracecmd_time_sync {
 };
 
 void tracecmd_tsync_init(void);
-int tracecmd_tsync_proto_getall(char **proto_mask, int *words);
-unsigned int tracecmd_tsync_proto_select(char *proto_mask, int words);
+int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock);
+unsigned int tracecmd_tsync_proto_select(char *proto_mask, int words, char *clock);
 bool tsync_proto_is_supported(unsigned int proto_id);
 void tracecmd_tsync_with_host(struct tracecmd_time_sync *tsync);
 void tracecmd_tsync_with_guest(struct tracecmd_time_sync *tsync);
diff --git a/lib/trace-cmd/include/trace-tsync-local.h b/lib/trace-cmd/include/trace-tsync-local.h
index 1de9d5e5..37c3597b 100644
--- a/lib/trace-cmd/include/trace-tsync-local.h
+++ b/lib/trace-cmd/include/trace-tsync-local.h
@@ -27,6 +27,7 @@ struct clock_sync_context {
 };
 
 int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
+				  int supported_clocks,
 				int (*init)(struct tracecmd_time_sync *),
 				int (*free)(struct tracecmd_time_sync *),
 				int (*calc)(struct tracecmd_time_sync *,
diff --git a/lib/trace-cmd/trace-timesync-ptp.c b/lib/trace-cmd/trace-timesync-ptp.c
index 19e2f647..51f17bca 100644
--- a/lib/trace-cmd/trace-timesync-ptp.c
+++ b/lib/trace-cmd/trace-timesync-ptp.c
@@ -678,6 +678,7 @@ int ptp_clock_sync_register(void)
 {
 	return tracecmd_tsync_proto_register(TRACECMD_TIME_SYNC_PROTO_PTP,
 					     TRACECMD_TIME_SYNC_PROTO_PTP_WEIGHT,
+					     0,
 					     ptp_clock_sync_init,
 					     ptp_clock_sync_free,
 					     ptp_clock_sync_calc);
diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
index e294698b..47d34e2a 100644
--- a/lib/trace-cmd/trace-timesync.c
+++ b/lib/trace-cmd/trace-timesync.c
@@ -26,6 +26,7 @@ struct tsync_proto {
 	struct tsync_proto *next;
 	unsigned int proto_id;
 	int	weight;
+	int supported_clocks;
 
 	int (*clock_sync_init)(struct tracecmd_time_sync *clock_context);
 	int (*clock_sync_free)(struct tracecmd_time_sync *clock_context);
@@ -55,6 +56,7 @@ void tracecmd_tsync_init(void)
 }
 
 int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
+				int supported_clocks,
 				int (*init)(struct tracecmd_time_sync *),
 				int (*free)(struct tracecmd_time_sync *),
 				int (*calc)(struct tracecmd_time_sync *,
@@ -69,6 +71,7 @@ int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
 		return -1;
 	proto->proto_id = proto_id;
 	proto->weight = weight;
+	proto->supported_clocks = supported_clocks;
 	proto->clock_sync_init = init;
 	proto->clock_sync_free = free;
 	proto->clock_sync_calc = calc;
@@ -139,19 +142,26 @@ int tracecmd_tsync_get_offsets(struct tracecmd_time_sync *tsync,
  *
  * @proto_mask: bitmask array of time sync protocols, supported by the peer
  * @length: size of the @protos array
+ * @clock: selected trace clock
  *
  * Retuns Id of a time sync protocol, that can be used with the peer, or 0
  *	  in case there is no match with supported protocols
  */
-unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length)
+unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length, char *clock)
 {
 	struct tsync_proto *selected = NULL;
 	struct tsync_proto *proto;
+	int clock_id = 0;
 	int word;
 	int id;
 
+	if (clock)
+		clock_id = tracecmd_clock_str2id(clock);
 	for (word = 0; word < length; word++) {
 		for (proto = tsync_proto_list; proto; proto = proto->next) {
+			if (proto->supported_clocks && clock_id &&
+			    !(proto->supported_clocks & clock_id))
+				continue;
 			if (proto->proto_id < word * PROTO_MASK_SIZE)
 				continue;
 
@@ -181,18 +191,22 @@ unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length)
  * @proto_mask: return, allocated bitmask array of time sync protocols,
  *	       supported by the peer. Must be freed by free()
  * @words: return, allocated size of the @protobits array
+ * @clock: selected trace clock
  *
  * If completed successfully 0 is returned and allocated array in @proto_mask of
  * size @words. In case of an error, -1 is returned.
  * @proto_mask must be freed with free()
  */
-int tracecmd_tsync_proto_getall(char **proto_mask, int *words)
+int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock)
 {
 	struct tsync_proto *proto;
 	int proto_max = 0;
+	int clock_id = 0;
 	int count = 0;
 	char *protos;
 
+	if (clock)
+		clock_id =  tracecmd_clock_str2id(clock);
 	for (proto = tsync_proto_list; proto; proto = proto->next)
 		if (proto->proto_id > proto_max)
 			proto_max = proto->proto_id;
@@ -205,6 +219,9 @@ int tracecmd_tsync_proto_getall(char **proto_mask, int *words)
 	for (proto = tsync_proto_list; proto; proto = proto->next) {
 		if ((proto->proto_id / PROTO_MASK_SIZE) >= count)
 			continue;
+		if (proto->supported_clocks && clock_id &&
+		    !(proto->supported_clocks & clock_id))
+			continue;
 		protos[proto->proto_id / PROTO_MASK_SIZE] |=
 				(1 << (proto->proto_id % PROTO_MASK_SIZE));
 	}
diff --git a/tracecmd/trace-record.c b/tracecmd/trace-record.c
index 9149cf76..08bbc0e0 100644
--- a/tracecmd/trace-record.c
+++ b/tracecmd/trace-record.c
@@ -3677,7 +3677,7 @@ static void connect_to_agent(struct buffer_instance *instance)
 		die("Failed to allocate message handle");
 
 	if (instance->tsync.loop_interval >= 0)
-		tracecmd_tsync_proto_getall(&protos, &protos_count);
+		tracecmd_tsync_proto_getall(&protos, &protos_count, instance->clock);
 
 	ret = tracecmd_msg_send_trace_req(msg_handle, instance->argc,
 					  instance->argv, use_fifos,
@@ -6111,6 +6111,8 @@ static void parse_record_options(int argc,
 						 (char *)top_instance.clock,
 						 true);
 					add_argv(instance, "-C", true);
+					if (!instance->clock)
+						instance->clock = strdup((char *)top_instance.clock);
 				}
 			}
 			instance->tsync.loop_interval = top_instance.tsync.loop_interval;
diff --git a/tracecmd/trace-tsync.c b/tracecmd/trace-tsync.c
index e639788d..832b70c3 100644
--- a/tracecmd/trace-tsync.c
+++ b/tracecmd/trace-tsync.c
@@ -220,7 +220,7 @@ unsigned int tracecmd_guest_tsync(char *tsync_protos,
 	int fd;
 
 	fd = -1;
-	proto = tracecmd_tsync_proto_select(tsync_protos, tsync_protos_size);
+	proto = tracecmd_tsync_proto_select(tsync_protos, tsync_protos_size, clock);
 	if (!proto)
 		return 0;
 #ifdef VSOCK
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 06/10] trace-cmd: Add role parameter to timestamp synchronization plugins
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (4 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 05/10] trace-cmd: Add clock parameter to timestamp synchronization plugins Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 07/10] trace-cmd: Add host / guest role in timestamp synchronization context Tzvetomir Stoyanov (VMware)
                   ` (3 subsequent siblings)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

Defined HOST and GUEST roles in timestamp synchronization context.
Some timestamp synchronization plugins may not support running in both
host and guest roles. This could happen in nested virtualization use
case, where the same plugin can be used as a host and as a guest.
Added logic to timestamp synchronization plugins to declare what roles
they support. Added logic to select plugin depending on supported roles
and currently requested role.

clocks. Added logic to timestamp synchronization plugins to declare what
ftace clocks they support. Added logic to select plugin depending on the
ftrace clock used in the current trace session and supported clocks.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 include/trace-cmd/trace-cmd.h             | 10 ++++++++--
 lib/trace-cmd/include/trace-tsync-local.h | 10 +++++-----
 lib/trace-cmd/trace-timesync-ptp.c        |  2 ++
 lib/trace-cmd/trace-timesync.c            | 16 ++++++++++++----
 tracecmd/trace-record.c                   |  3 ++-
 tracecmd/trace-tsync.c                    |  3 ++-
 6 files changed, 31 insertions(+), 13 deletions(-)

diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
index 66b5d02b..f1068c49 100644
--- a/include/trace-cmd/trace-cmd.h
+++ b/include/trace-cmd/trace-cmd.h
@@ -442,6 +442,11 @@ enum{
 	TRACECMD_TIME_SYNC_CMD_STOP	= 2,
 };
 
+enum tracecmd_time_sync_role {
+	TRACECMD_TIME_SYNC_ROLE_HOST	= 0x01,
+	TRACECMD_TIME_SYNC_ROLE_GUEST	= 0x02
+};
+
 #define TRACECMD_TIME_SYNC_PROTO_PTP_WEIGHT	10
 
 struct tracecmd_time_sync {
@@ -455,8 +460,9 @@ struct tracecmd_time_sync {
 };
 
 void tracecmd_tsync_init(void);
-int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock);
-unsigned int tracecmd_tsync_proto_select(char *proto_mask, int words, char *clock);
+int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock, int role);
+unsigned int tracecmd_tsync_proto_select(char *proto_mask, int words, char *clock,
+					 enum tracecmd_time_sync_role role);
 bool tsync_proto_is_supported(unsigned int proto_id);
 void tracecmd_tsync_with_host(struct tracecmd_time_sync *tsync);
 void tracecmd_tsync_with_guest(struct tracecmd_time_sync *tsync);
diff --git a/lib/trace-cmd/include/trace-tsync-local.h b/lib/trace-cmd/include/trace-tsync-local.h
index 37c3597b..b79b102c 100644
--- a/lib/trace-cmd/include/trace-tsync-local.h
+++ b/lib/trace-cmd/include/trace-tsync-local.h
@@ -26,12 +26,12 @@ struct clock_sync_context {
 	unsigned int			remote_port;
 };
 
-int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
+int tracecmd_tsync_proto_register(unsigned int proto_id, int weight, int roles,
 				  int supported_clocks,
-				int (*init)(struct tracecmd_time_sync *),
-				int (*free)(struct tracecmd_time_sync *),
-				int (*calc)(struct tracecmd_time_sync *,
-					    long long *, long long *));
+				  int (*init)(struct tracecmd_time_sync *),
+				  int (*free)(struct tracecmd_time_sync *),
+				  int (*calc)(struct tracecmd_time_sync *,
+					      long long *, long long *));
 int tracecmd_tsync_proto_unregister(unsigned int proto_id);
 
 int ptp_clock_sync_register(void);
diff --git a/lib/trace-cmd/trace-timesync-ptp.c b/lib/trace-cmd/trace-timesync-ptp.c
index 51f17bca..07ce3b6e 100644
--- a/lib/trace-cmd/trace-timesync-ptp.c
+++ b/lib/trace-cmd/trace-timesync-ptp.c
@@ -678,6 +678,8 @@ int ptp_clock_sync_register(void)
 {
 	return tracecmd_tsync_proto_register(TRACECMD_TIME_SYNC_PROTO_PTP,
 					     TRACECMD_TIME_SYNC_PROTO_PTP_WEIGHT,
+					     TRACECMD_TIME_SYNC_ROLE_GUEST |
+					     TRACECMD_TIME_SYNC_ROLE_HOST,
 					     0,
 					     ptp_clock_sync_init,
 					     ptp_clock_sync_free,
diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
index 47d34e2a..807fb6f5 100644
--- a/lib/trace-cmd/trace-timesync.c
+++ b/lib/trace-cmd/trace-timesync.c
@@ -25,6 +25,7 @@
 struct tsync_proto {
 	struct tsync_proto *next;
 	unsigned int proto_id;
+	enum tracecmd_time_sync_role roles;
 	int	weight;
 	int supported_clocks;
 
@@ -55,7 +56,7 @@ void tracecmd_tsync_init(void)
 	ptp_clock_sync_register();
 }
 
-int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
+int tracecmd_tsync_proto_register(unsigned int proto_id, int weight, int roles,
 				int supported_clocks,
 				int (*init)(struct tracecmd_time_sync *),
 				int (*free)(struct tracecmd_time_sync *),
@@ -71,6 +72,7 @@ int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
 		return -1;
 	proto->proto_id = proto_id;
 	proto->weight = weight;
+	proto->roles = roles;
 	proto->supported_clocks = supported_clocks;
 	proto->clock_sync_init = init;
 	proto->clock_sync_free = free;
@@ -142,12 +144,13 @@ int tracecmd_tsync_get_offsets(struct tracecmd_time_sync *tsync,
  *
  * @proto_mask: bitmask array of time sync protocols, supported by the peer
  * @length: size of the @protos array
- * @clock: selected trace clock
+ * @role : local time sync role
  *
  * Retuns Id of a time sync protocol, that can be used with the peer, or 0
  *	  in case there is no match with supported protocols
  */
-unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length, char *clock)
+unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length, char *clock,
+					 enum tracecmd_time_sync_role role)
 {
 	struct tsync_proto *selected = NULL;
 	struct tsync_proto *proto;
@@ -159,6 +162,8 @@ unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length, char *clo
 		clock_id = tracecmd_clock_str2id(clock);
 	for (word = 0; word < length; word++) {
 		for (proto = tsync_proto_list; proto; proto = proto->next) {
+			if (!(proto->roles & role))
+				continue;
 			if (proto->supported_clocks && clock_id &&
 			    !(proto->supported_clocks & clock_id))
 				continue;
@@ -192,12 +197,13 @@ unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length, char *clo
  *	       supported by the peer. Must be freed by free()
  * @words: return, allocated size of the @protobits array
  * @clock: selected trace clock
+ * @role: supported protocol role
  *
  * If completed successfully 0 is returned and allocated array in @proto_mask of
  * size @words. In case of an error, -1 is returned.
  * @proto_mask must be freed with free()
  */
-int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock)
+int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock, int role)
 {
 	struct tsync_proto *proto;
 	int proto_max = 0;
@@ -219,6 +225,8 @@ int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock
 	for (proto = tsync_proto_list; proto; proto = proto->next) {
 		if ((proto->proto_id / PROTO_MASK_SIZE) >= count)
 			continue;
+		if (!(proto->roles & role))
+			continue;
 		if (proto->supported_clocks && clock_id &&
 		    !(proto->supported_clocks & clock_id))
 			continue;
diff --git a/tracecmd/trace-record.c b/tracecmd/trace-record.c
index 08bbc0e0..ff70df43 100644
--- a/tracecmd/trace-record.c
+++ b/tracecmd/trace-record.c
@@ -3677,7 +3677,8 @@ static void connect_to_agent(struct buffer_instance *instance)
 		die("Failed to allocate message handle");
 
 	if (instance->tsync.loop_interval >= 0)
-		tracecmd_tsync_proto_getall(&protos, &protos_count, instance->clock);
+		tracecmd_tsync_proto_getall(&protos, &protos_count, instance->clock,
+					    TRACECMD_TIME_SYNC_ROLE_HOST);
 
 	ret = tracecmd_msg_send_trace_req(msg_handle, instance->argc,
 					  instance->argv, use_fifos,
diff --git a/tracecmd/trace-tsync.c b/tracecmd/trace-tsync.c
index 832b70c3..051b1003 100644
--- a/tracecmd/trace-tsync.c
+++ b/tracecmd/trace-tsync.c
@@ -220,7 +220,8 @@ unsigned int tracecmd_guest_tsync(char *tsync_protos,
 	int fd;
 
 	fd = -1;
-	proto = tracecmd_tsync_proto_select(tsync_protos, tsync_protos_size, clock);
+	proto = tracecmd_tsync_proto_select(tsync_protos, tsync_protos_size,
+					    clock, TRACECMD_TIME_SYNC_ROLE_GUEST);
 	if (!proto)
 		return 0;
 #ifdef VSOCK
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 07/10] trace-cmd: Add host / guest role in timestamp synchronization context
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (5 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 06/10] trace-cmd: Add role " Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 08/10] trace-cmd: Add guest CPU count PID in tracecmd_time_sync struct Tzvetomir Stoyanov (VMware)
                   ` (2 subsequent siblings)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

Added new parameter in timestamp synchronization context, holding the
current role in the timestamp synchronization process - host or guest.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 lib/trace-cmd/include/trace-tsync-local.h | 1 +
 lib/trace-cmd/trace-timesync.c            | 9 ++++++---
 2 files changed, 7 insertions(+), 3 deletions(-)

diff --git a/lib/trace-cmd/include/trace-tsync-local.h b/lib/trace-cmd/include/trace-tsync-local.h
index b79b102c..1af52a12 100644
--- a/lib/trace-cmd/include/trace-tsync-local.h
+++ b/lib/trace-cmd/include/trace-tsync-local.h
@@ -11,6 +11,7 @@
 struct clock_sync_context {
 	void				*proto_data;	/* time sync protocol specific data */
 	bool				is_server;	/* server side time sync role */
+	bool				is_guest;	/* guest or host time sync role */
 	struct tracefs_instance		*instance;	/* ftrace buffer, used for time sync events */
 
 	/* Arrays with calculated time offsets at given time */
diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
index 807fb6f5..10ab82f0 100644
--- a/lib/trace-cmd/trace-timesync.c
+++ b/lib/trace-cmd/trace-timesync.c
@@ -293,7 +293,7 @@ clock_synch_delete_instance(struct tracefs_instance *inst)
 	tracefs_instance_free(inst);
 }
 
-static int clock_context_init(struct tracecmd_time_sync *tsync, bool server)
+static int clock_context_init(struct tracecmd_time_sync *tsync, bool guest)
 {
 	struct clock_sync_context *clock = NULL;
 	struct tsync_proto *protocol;
@@ -308,8 +308,11 @@ static int clock_context_init(struct tracecmd_time_sync *tsync, bool server)
 	clock = calloc(1, sizeof(struct clock_sync_context));
 	if (!clock)
 		return -1;
-
-	clock->is_server = server;
+	clock->is_guest = guest;
+	if (clock->is_guest)
+		clock->is_server = true;
+	else
+		clock->is_server = false;
 	if (get_vsocket_params(tsync->msg_handle->fd, &clock->local_cid,
 			       &clock->local_port, &clock->remote_cid,
 			       &clock->remote_port))
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 08/10] trace-cmd: Add guest CPU count PID in tracecmd_time_sync struct
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (6 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 07/10] trace-cmd: Add host / guest role in timestamp synchronization context Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 09/10] trace-cmd: Fixed bitmask logic tracecmd_tsync_proto_getall() Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 10/10] trace-cmd [POC]: Add KVM timestamp synchronization plugin Tzvetomir Stoyanov (VMware)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

The tracecmd_time_sync struct holds the timestamp synchronization
context, used by the timestamp synchronization plugins. Guest CPU count and
PID of the host task, running the guest, is important information which
may be needed by the plugins.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 include/trace-cmd/trace-cmd.h | 4 ++++
 tracecmd/trace-tsync.c        | 7 ++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
index f1068c49..48ed4bc0 100644
--- a/include/trace-cmd/trace-cmd.h
+++ b/include/trace-cmd/trace-cmd.h
@@ -457,6 +457,10 @@ struct tracecmd_time_sync {
 	char				*clock_str;
 	struct tracecmd_msg_handle	*msg_handle;
 	void				*context;
+
+	int				guest_pid;
+	int				vcpu_count;
+
 };
 
 void tracecmd_tsync_init(void);
diff --git a/tracecmd/trace-tsync.c b/tracecmd/trace-tsync.c
index 051b1003..27718172 100644
--- a/tracecmd/trace-tsync.c
+++ b/tracecmd/trace-tsync.c
@@ -77,6 +77,7 @@ int tracecmd_host_tsync(struct buffer_instance *instance,
 {
 	struct tracecmd_msg_handle *msg_handle = NULL;
 	cpu_set_t *pin_mask = NULL;
+	struct trace_guest *guest;
 	pthread_attr_t attrib;
 	size_t mask_size = 0;
 	int ret;
@@ -84,7 +85,11 @@ int tracecmd_host_tsync(struct buffer_instance *instance,
 
 	if (!instance->tsync.sync_proto)
 		return -1;
-
+	guest = get_guest_by_cid(instance->cid);
+	if (guest == NULL)
+		return -1;
+	instance->tsync.guest_pid = guest->pid;
+	instance->tsync.vcpu_count = guest->cpu_max;
 	fd = trace_open_vsock(instance->cid, tsync_port);
 	if (fd < 0) {
 		ret = -1;
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 09/10] trace-cmd: Fixed bitmask logic tracecmd_tsync_proto_getall()
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (7 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 08/10] trace-cmd: Add guest CPU count PID in tracecmd_time_sync struct Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  2020-10-09 14:03 ` [PATCH v24 10/10] trace-cmd [POC]: Add KVM timestamp synchronization plugin Tzvetomir Stoyanov (VMware)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

The tracecmd_tsync_proto_getall () function returns a bitmask of all
registered timestamp synchronization plugins, which match given criteria.
There was a bug in that logic - it was working with wrong bitmask size,
 which leads to broken bitmask.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 lib/trace-cmd/trace-timesync.c | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
index 10ab82f0..2fbb319b 100644
--- a/lib/trace-cmd/trace-timesync.c
+++ b/lib/trace-cmd/trace-timesync.c
@@ -217,21 +217,21 @@ int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock
 		if (proto->proto_id > proto_max)
 			proto_max = proto->proto_id;
 
-	count = proto_max / PROTO_MASK_SIZE + 1;
-	protos = calloc(count, sizeof(char));
+	count = proto_max / PROTO_MASK_BITS + 1;
+	protos = calloc(count, PROTO_MASK_SIZE);
 	if (!protos)
 		return -1;
 
 	for (proto = tsync_proto_list; proto; proto = proto->next) {
-		if ((proto->proto_id / PROTO_MASK_SIZE) >= count)
+		if ((proto->proto_id / PROTO_MASK_BITS) >= count)
 			continue;
 		if (!(proto->roles & role))
 			continue;
 		if (proto->supported_clocks && clock_id &&
 		    !(proto->supported_clocks & clock_id))
 			continue;
-		protos[proto->proto_id / PROTO_MASK_SIZE] |=
-				(1 << (proto->proto_id % PROTO_MASK_SIZE));
+		protos[proto->proto_id / PROTO_MASK_BITS] |=
+				(1 << (proto->proto_id % PROTO_MASK_BITS));
 	}
 
 	*proto_mask = protos;
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* [PATCH v24 10/10] trace-cmd [POC]: Add KVM timestamp synchronization plugin
  2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
                   ` (8 preceding siblings ...)
  2020-10-09 14:03 ` [PATCH v24 09/10] trace-cmd: Fixed bitmask logic tracecmd_tsync_proto_getall() Tzvetomir Stoyanov (VMware)
@ 2020-10-09 14:03 ` Tzvetomir Stoyanov (VMware)
  9 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov (VMware) @ 2020-10-09 14:03 UTC (permalink / raw)
  To: rostedt; +Cc: linux-trace-devel

Added new timestamp synchronization plugin for KVM hosts. It reads the
clock offsets directly from the KVM debug filesystem, if available.
The plugin works only with x86-tsc ftrace clock.

Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
---
 include/trace-cmd/trace-cmd.h             |   2 +
 lib/trace-cmd/Makefile                    |   1 +
 lib/trace-cmd/include/trace-tsync-local.h |   1 +
 lib/trace-cmd/trace-timesync-kvm.c        | 427 ++++++++++++++++++++++
 lib/trace-cmd/trace-timesync.c            |   1 +
 5 files changed, 432 insertions(+)
 create mode 100644 lib/trace-cmd/trace-timesync-kvm.c

diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
index 48ed4bc0..67c7a42f 100644
--- a/include/trace-cmd/trace-cmd.h
+++ b/include/trace-cmd/trace-cmd.h
@@ -436,6 +436,7 @@ char *tracecmd_clock_id2str(enum tracecmd_clocks clock);
 enum{
 	TRACECMD_TIME_SYNC_PROTO_NONE	= 0,
 	TRACECMD_TIME_SYNC_PROTO_PTP	= 1,
+	TRACECMD_TIME_SYNC_PROTO_KVM	= 2,
 };
 enum{
 	TRACECMD_TIME_SYNC_CMD_PROBE	= 1,
@@ -447,6 +448,7 @@ enum tracecmd_time_sync_role {
 	TRACECMD_TIME_SYNC_ROLE_GUEST	= 0x02
 };
 
+#define TRACECMD_TIME_SYNC_PROTO_KVM_WEIGHT	100
 #define TRACECMD_TIME_SYNC_PROTO_PTP_WEIGHT	10
 
 struct tracecmd_time_sync {
diff --git a/lib/trace-cmd/Makefile b/lib/trace-cmd/Makefile
index 7cab7514..d1b8fc49 100644
--- a/lib/trace-cmd/Makefile
+++ b/lib/trace-cmd/Makefile
@@ -19,6 +19,7 @@ OBJS += trace-plugin.o
 ifeq ($(VSOCK_DEFINED), 1)
 OBJS += trace-timesync.o
 OBJS += trace-timesync-ptp.o
+OBJS += trace-timesync-kvm.o
 endif
 
 # Additional util objects
diff --git a/lib/trace-cmd/include/trace-tsync-local.h b/lib/trace-cmd/include/trace-tsync-local.h
index 1af52a12..b75dfe01 100644
--- a/lib/trace-cmd/include/trace-tsync-local.h
+++ b/lib/trace-cmd/include/trace-tsync-local.h
@@ -36,5 +36,6 @@ int tracecmd_tsync_proto_register(unsigned int proto_id, int weight, int roles,
 int tracecmd_tsync_proto_unregister(unsigned int proto_id);
 
 int ptp_clock_sync_register(void);
+int kvm_clock_sync_register(void);
 
 #endif /* _TRACE_TSYNC_LOCAL_H */
diff --git a/lib/trace-cmd/trace-timesync-kvm.c b/lib/trace-cmd/trace-timesync-kvm.c
new file mode 100644
index 00000000..ecf0a880
--- /dev/null
+++ b/lib/trace-cmd/trace-timesync-kvm.c
@@ -0,0 +1,427 @@
+// SPDX-License-Identifier: LGPL-2.1
+/*
+ * Copyright (C) 2020, VMware, Tzvetomir Stoyanov tz.stoyanov@gmail.com>
+ *
+ */
+
+#include <fcntl.h>
+#include <stdlib.h>
+#include <unistd.h>
+#include <sys/stat.h>
+#include <dirent.h>
+
+#include "trace-cmd.h"
+#include "tracefs.h"
+#include "trace-tsync-local.h"
+
+#define KVM_DEBUG_FS "/sys/kernel/debug/kvm"
+#define KVM_DEBUG_OFFSET_FILE	"tsc-offset"
+#define KVM_DEBUG_SCALING_FILE	"tsc-scaling-ratio"
+#define KVM_DEBUG_VCPU_DIR	"vcpu"
+
+#define KVM_SYNC_PKT_REQUEST	1
+#define KVM_SYNC_PKT_RESPONSE	2
+
+typedef __s64 s64;
+
+struct kvm_clock_sync {
+	int vcpu_count;
+	char **vcpu_offsets;
+	char **vcpu_scalings;
+	int marker_fd;
+	struct tep_handle *tep;
+	int raw_id;
+	unsigned long long ts;
+};
+
+struct kvm_clock_offset_msg {
+	s64	ts;
+	s64	offset;
+};
+
+static bool kvm_support_check(bool guest)
+{
+	struct stat st;
+	int ret;
+
+	if (guest)
+		return true;
+
+	ret = stat(KVM_DEBUG_FS, &st);
+	if (ret < 0)
+		return false;
+
+	if (!S_ISDIR(st.st_mode))
+		return false;
+	return true;
+}
+
+static int kvm_open_vcpu_dir(struct kvm_clock_sync *kvm, int cpu, char *dir_str)
+{
+	struct dirent *entry;
+	char path[PATH_MAX];
+	DIR *dir;
+
+	dir = opendir(dir_str);
+	if (!dir)
+		goto error;
+	while ((entry = readdir(dir))) {
+		if (entry->d_type != DT_DIR) {
+			if (!strncmp(entry->d_name, KVM_DEBUG_OFFSET_FILE,
+				     strlen(KVM_DEBUG_OFFSET_FILE))) {
+				snprintf(path, sizeof(path), "%s/%s",
+					 dir_str, entry->d_name);
+				kvm->vcpu_offsets[cpu] = strdup(path);
+			}
+			if (!strncmp(entry->d_name, KVM_DEBUG_SCALING_FILE,
+				     strlen(KVM_DEBUG_SCALING_FILE))) {
+				snprintf(path, sizeof(path), "%s/%s",
+					 dir_str, entry->d_name);
+				kvm->vcpu_scalings[cpu] = strdup(path);
+			}
+		}
+	}
+	if (!kvm->vcpu_offsets[cpu])
+		goto error;
+	closedir(dir);
+	return 0;
+
+error:
+	if (dir)
+		closedir(dir);
+	free(kvm->vcpu_offsets[cpu]);
+	kvm->vcpu_offsets[cpu] = NULL;
+	free(kvm->vcpu_scalings[cpu]);
+	kvm->vcpu_scalings[cpu] = NULL;
+	return -1;
+}
+
+static int kvm_open_debug_files(struct kvm_clock_sync *kvm, int pid)
+{
+	char *vm_dir_str = NULL;
+	struct dirent *entry;
+	char *pid_str = NULL;
+	char path[PATH_MAX];
+	long vcpu;
+	DIR *dir;
+	int i;
+
+	dir = opendir(KVM_DEBUG_FS);
+	if (!dir)
+		goto error;
+	if (asprintf(&pid_str, "%d-", pid) <= 0)
+		goto error;
+	while ((entry = readdir(dir))) {
+		if (!(entry->d_type == DT_DIR &&
+		    !strncmp(entry->d_name, pid_str, strlen(pid_str))))
+			continue;
+		asprintf(&vm_dir_str, "%s/%s", KVM_DEBUG_FS, entry->d_name);
+		break;
+	}
+	closedir(dir);
+	dir = NULL;
+	if (!vm_dir_str)
+		goto error;
+	dir = opendir(vm_dir_str);
+	if (!dir)
+		goto error;
+	while ((entry = readdir(dir))) {
+		if (!(entry->d_type == DT_DIR &&
+		    !strncmp(entry->d_name, KVM_DEBUG_VCPU_DIR, strlen(KVM_DEBUG_VCPU_DIR))))
+			continue;
+		vcpu =  strtol(entry->d_name + strlen(KVM_DEBUG_VCPU_DIR), NULL, 10);
+		if (vcpu < 0 || vcpu >= kvm->vcpu_count)
+			continue;
+		snprintf(path, sizeof(path), "%s/%s", vm_dir_str, entry->d_name);
+		if (kvm_open_vcpu_dir(kvm, vcpu, path) < 0)
+			goto error;
+	}
+	for (i = 0; i < kvm->vcpu_count; i++) {
+		if (!kvm->vcpu_offsets[i])
+			goto error;
+	}
+	closedir(dir);
+	free(pid_str);
+	free(vm_dir_str);
+	return 0;
+error:
+	free(pid_str);
+	free(vm_dir_str);
+	if (dir)
+		closedir(dir);
+	return -1;
+}
+
+static int kvm_clock_sync_init_host(struct tracecmd_time_sync *tsync,
+				    struct kvm_clock_sync *kvm)
+{
+	kvm->vcpu_count = tsync->vcpu_count;
+	kvm->vcpu_offsets = calloc(kvm->vcpu_count, sizeof(char *));
+	kvm->vcpu_scalings = calloc(kvm->vcpu_count, sizeof(char *));
+	if (!kvm->vcpu_offsets || !kvm->vcpu_scalings)
+		goto error;
+	if (kvm_open_debug_files(kvm, tsync->guest_pid) < 0)
+		goto error;
+	return 0;
+
+error:
+	free(kvm->vcpu_offsets);
+	free(kvm->vcpu_scalings);
+	return -1;
+}
+
+static int kvm_clock_sync_init_guest(struct tracecmd_time_sync *tsync,
+				     struct kvm_clock_sync *kvm)
+{
+	const char *systems[] = {"ftrace", NULL};
+	struct clock_sync_context *clock_context;
+	struct tep_event *raw;
+	char *path;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+	path = tracefs_instance_get_dir(clock_context->instance);
+	if (!path)
+		goto error;
+	kvm->tep = tracefs_local_events_system(path, systems);
+	tracefs_put_tracing_file(path);
+	if (!kvm->tep)
+		goto error;
+	raw = tep_find_event_by_name(kvm->tep, "ftrace", "raw_data");
+	if (!raw)
+		goto error;
+
+	kvm->raw_id = raw->id;
+	tep_set_file_bigendian(kvm->tep, tracecmd_host_bigendian());
+	tep_set_local_bigendian(kvm->tep, tracecmd_host_bigendian());
+
+	path = tracefs_instance_get_file(clock_context->instance, "trace_marker_raw");
+	if (!path)
+		goto error;
+	kvm->marker_fd = open(path, O_WRONLY);
+	tracefs_put_tracing_file(path);
+
+	return 0;
+
+error:
+	if (kvm->tep)
+		tep_free(kvm->tep);
+	if (kvm->marker_fd >= 0)
+		close(kvm->marker_fd);
+
+	return -1;
+}
+
+static int kvm_clock_sync_init(struct tracecmd_time_sync *tsync)
+{
+	struct clock_sync_context *clock_context;
+	struct kvm_clock_sync *kvm;
+	int ret;
+
+	if (!tsync || !tsync->context)
+		return -1;
+	clock_context = (struct clock_sync_context *)tsync->context;
+
+	if (!kvm_support_check(clock_context->is_guest))
+		return -1;
+	kvm = calloc(1, sizeof(struct kvm_clock_sync));
+	if (!kvm)
+		return -1;
+	kvm->marker_fd = -1;
+	if (clock_context->is_guest)
+		ret = kvm_clock_sync_init_guest(tsync, kvm);
+	else
+		ret = kvm_clock_sync_init_host(tsync, kvm);
+	if (ret < 0)
+		goto error;
+
+	clock_context->proto_data = kvm;
+	return 0;
+
+error:
+	free(kvm);
+	return -1;
+}
+
+static int kvm_clock_sync_free(struct tracecmd_time_sync *tsync)
+{
+	struct clock_sync_context *clock_context;
+	struct kvm_clock_sync *kvm = NULL;
+	int i;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+	if (clock_context)
+		kvm = (struct kvm_clock_sync *)clock_context->proto_data;
+	if (kvm) {
+		for (i = 0; i < kvm->vcpu_count; i++) {
+			free(kvm->vcpu_offsets[i]);
+			kvm->vcpu_offsets[i] = NULL;
+			free(kvm->vcpu_scalings[i]);
+			kvm->vcpu_scalings[i] = NULL;
+		}
+		if (kvm->tep)
+			tep_free(kvm->tep);
+		if (kvm->marker_fd >= 0)
+			close(kvm->marker_fd);
+		free(kvm);
+	}
+	return -1;
+}
+
+static int kvm_clock_host(struct tracecmd_time_sync *tsync,
+			  long long *offset, long long *timestamp)
+{
+	struct clock_sync_context *clock_context;
+	struct kvm_clock_offset_msg packet;
+	struct kvm_clock_sync *kvm = NULL;
+	unsigned int sync_proto, sync_msg;
+	unsigned int size;
+	char buf[32];
+	char *msg;
+	int ret;
+	int fd;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+	if (clock_context)
+		kvm = (struct kvm_clock_sync *)clock_context->proto_data;
+	if (!kvm || !kvm->vcpu_offsets || !kvm->vcpu_offsets[0])
+		return -1;
+	fd = open(kvm->vcpu_offsets[0], O_RDONLY | O_NONBLOCK);
+	if (fd < 0)
+		return -1;
+	ret = read(fd, buf, 32);
+	close(fd);
+	if (ret <= 0)
+		return -1;
+
+	msg = (char *)&packet;
+	size = sizeof(packet);
+	ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+					  &sync_proto, &sync_msg,
+					  &size, &msg);
+	if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_KVM ||
+	    sync_msg != KVM_SYNC_PKT_REQUEST)
+		return -1;
+
+	packet.offset = -strtoll(buf, NULL, 10);
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					  TRACECMD_TIME_SYNC_PROTO_KVM,
+					  KVM_SYNC_PKT_RESPONSE, sizeof(packet),
+					  (char *)&packet);
+	if (ret)
+		return -1;
+
+	*offset = packet.offset;
+	*timestamp = packet.ts;
+	return 0;
+}
+
+#define KVM_EVENT_MARKER	"kvm sync event"
+static int kvm_marker_find(struct tep_event *event, struct tep_record *record,
+			   int cpu, void *context)
+{
+	struct kvm_clock_sync *kvm = (struct kvm_clock_sync *)context;
+	struct tep_format_field *field;
+	struct tep_format_field *id;
+	char *marker;
+
+	/* Make sure this is our event */
+	if (event->id != kvm->raw_id)
+		return 0;
+	id = tep_find_field(event, "id");
+	field = tep_find_field(event, "buf");
+	if (field && id &&
+	    record->size >= (id->offset + strlen(KVM_EVENT_MARKER) + 1)) {
+		marker = (char *)(record->data + id->offset);
+		if (!strcmp(marker, KVM_EVENT_MARKER)) {
+			kvm->ts = record->ts;
+			return 1;
+		}
+	}
+
+	return 0;
+}
+
+
+static int kvm_clock_guest(struct tracecmd_time_sync *tsync,
+			   long long *offset, long long *timestamp)
+{
+	struct clock_sync_context *clock_context;
+	struct kvm_clock_offset_msg packet;
+	struct kvm_clock_sync *kvm = NULL;
+	unsigned int sync_proto, sync_msg;
+	unsigned int size;
+	char *msg;
+	int ret;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+	if (clock_context)
+		kvm = (struct kvm_clock_sync *)clock_context->proto_data;
+	if (!kvm)
+		return -1;
+	kvm->ts = 0;
+	memset(&packet, 0, sizeof(packet));
+	tracefs_instance_file_write(clock_context->instance, "trace", "\0");
+	write(kvm->marker_fd, KVM_EVENT_MARKER, strlen(KVM_EVENT_MARKER) + 1);
+	kvm->ts = 0;
+	tracefs_iterate_raw_events(kvm->tep, clock_context->instance,
+				   kvm_marker_find, kvm);
+	packet.ts = kvm->ts;
+	ret = tracecmd_msg_send_time_sync(tsync->msg_handle,
+					  TRACECMD_TIME_SYNC_PROTO_KVM,
+					  KVM_SYNC_PKT_REQUEST, sizeof(packet),
+					  (char *)&packet);
+	if (ret)
+		return -1;
+	msg = (char *)&packet;
+	size = sizeof(packet);
+	ret = tracecmd_msg_recv_time_sync(tsync->msg_handle,
+					  &sync_proto, &sync_msg,
+					  &size, &msg);
+	if (ret || sync_proto != TRACECMD_TIME_SYNC_PROTO_KVM ||
+	    sync_msg != KVM_SYNC_PKT_RESPONSE)
+		return -1;
+
+	*offset = packet.offset;
+	*timestamp = packet.ts;
+	return 0;
+}
+
+static int kvm_clock_sync_calc(struct tracecmd_time_sync *tsync,
+			       long long *offset, long long *timestamp)
+{
+	struct clock_sync_context *clock_context;
+	int ret;
+
+	if (!tsync || !tsync->context)
+		return -1;
+
+	clock_context = (struct clock_sync_context *)tsync->context;
+
+	if (clock_context->is_guest)
+		ret = kvm_clock_guest(tsync, offset, timestamp);
+	else
+		ret = kvm_clock_host(tsync, offset, timestamp);
+	return ret;
+}
+
+int kvm_clock_sync_register(void)
+{
+	int role = TRACECMD_TIME_SYNC_ROLE_GUEST;
+	int clock = 0;
+
+	if (kvm_support_check(false)) {
+		role |= TRACECMD_TIME_SYNC_ROLE_HOST;
+		clock = TRACECMD_CLOCK_X86_TSC;
+	}
+	return tracecmd_tsync_proto_register(TRACECMD_TIME_SYNC_PROTO_KVM,
+					     TRACECMD_TIME_SYNC_PROTO_KVM_WEIGHT,
+					     role, clock,
+					     kvm_clock_sync_init,
+					     kvm_clock_sync_free,
+					     kvm_clock_sync_calc);
+}
+
+int kvm_clock_sync_unregister(void)
+{
+	return tracecmd_tsync_proto_unregister(TRACECMD_TIME_SYNC_PROTO_KVM);
+}
diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
index 2fbb319b..e47f484b 100644
--- a/lib/trace-cmd/trace-timesync.c
+++ b/lib/trace-cmd/trace-timesync.c
@@ -54,6 +54,7 @@ static struct tsync_proto *tsync_proto_find(unsigned int proto_id)
 void tracecmd_tsync_init(void)
 {
 	ptp_clock_sync_register();
+	kvm_clock_sync_register();
 }
 
 int tracecmd_tsync_proto_register(unsigned int proto_id, int weight, int roles,
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 16+ messages in thread

* Re: [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization
  2020-10-09 14:03 ` [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization Tzvetomir Stoyanov (VMware)
@ 2020-10-15 21:24   ` Steven Rostedt
  2020-10-26  7:38     ` Tzvetomir Stoyanov
  0 siblings, 1 reply; 16+ messages in thread
From: Steven Rostedt @ 2020-10-15 21:24 UTC (permalink / raw)
  To: Tzvetomir Stoyanov (VMware); +Cc: linux-trace-devel

On Fri,  9 Oct 2020 17:03:29 +0300
"Tzvetomir Stoyanov (VMware)" <tz.stoyanov@gmail.com> wrote:

> PTP protocol is designed for synchronizing clocks of machines in a local network.
> The same approach can be used for host - guest timestamp synchronization.
> This implementation uses ftrace raw markers to track trace timestamps of PTP events.
> The patch is a POC, two different algorithms for PTP calculations are proposed:
>   - Choosing the sample with the fastest response time for calculating the clocks offset.
>   - Calculating the clocks offset using the average of all PTP samples.
> 
> The implementation can be tuned using those parameters:
>  - #define FASTEST_RESPONSE - is defined, the sample with the fastest response time
>     is used for calculating the clocks offset. Otherwise the histogram of all samples is used.
>  - #define PTP_SYNC_LOOP 340 - defines the number of samples, used for one calculation.
>  - --tsync-interval - a trace-cmd argument, choose the intervals between offset calculations,
> 	performed continuously during the trace.
>  - #define TSYNC_DEBUG - if defined, a debug information is collected and stored in files,
>    in the guest machine:
>      s-cid*.txt - For each offset calculation: host and guest clocks and calculated offset.
>      res-cid*.txt - For each tracing session: all calculated clock offsets.
> 
> Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
> ---
>  include/trace-cmd/trace-cmd.h      |   1 +
>  lib/trace-cmd/Makefile             |   1 +
>  lib/trace-cmd/trace-timesync-ptp.c | 690 +++++++++++++++++++++++++++++
>  lib/trace-cmd/trace-timesync.c     |   8 +
>  tracecmd/trace-agent.c             |   2 +
>  tracecmd/trace-record.c            |  23 +-
>  6 files changed, 717 insertions(+), 8 deletions(-)
>  create mode 100644 lib/trace-cmd/trace-timesync-ptp.c
> 
> diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
> index f3c95f30..f9c1f843 100644
> --- a/include/trace-cmd/trace-cmd.h
> +++ b/include/trace-cmd/trace-cmd.h
> @@ -419,6 +419,7 @@ int tracecmd_msg_recv_time_sync(struct tracecmd_msg_handle *msg_handle,
>  
>  enum{
>  	TRACECMD_TIME_SYNC_PROTO_NONE	= 0,
> +	TRACECMD_TIME_SYNC_PROTO_PTP	= 1,
>  };
>  enum{
>  	TRACECMD_TIME_SYNC_CMD_PROBE	= 1,
> diff --git a/lib/trace-cmd/Makefile b/lib/trace-cmd/Makefile
> index 666a1ebf..7cab7514 100644
> --- a/lib/trace-cmd/Makefile
> +++ b/lib/trace-cmd/Makefile
> @@ -18,6 +18,7 @@ OBJS += trace-msg.o
>  OBJS += trace-plugin.o
>  ifeq ($(VSOCK_DEFINED), 1)
>  OBJS += trace-timesync.o
> +OBJS += trace-timesync-ptp.o
>  endif
>  
>  # Additional util objects
> diff --git a/lib/trace-cmd/trace-timesync-ptp.c b/lib/trace-cmd/trace-timesync-ptp.c
> new file mode 100644
> index 00000000..19e2f647
> --- /dev/null
> +++ b/lib/trace-cmd/trace-timesync-ptp.c
> @@ -0,0 +1,690 @@
> +// SPDX-License-Identifier: LGPL-2.1
> +/*
> + * Copyright (C) 2019, VMware, Tzvetomir Stoyanov tz.stoyanov@gmail.com>
> + *
> + */
> +
> +#include <fcntl.h>
> +#include <stdlib.h>
> +#include <unistd.h>
> +#include <arpa/inet.h>
> +#include <linux/vm_sockets.h>
> +#include <sys/types.h>
> +#include <linux/types.h>
> +#include <time.h>
> +#include <sched.h>
> +#include "trace-cmd.h"
> +#include "tracefs.h"
> +#include "trace-tsync-local.h"
> +#include "trace-msg.h"
> +#include "trace-cmd-local.h"
> +
> +typedef __be32 be32;
> +typedef __u64 u64;
> +typedef __s64 s64;
> +
> +#define PTP_SYNC_LOOP	340
> +
> +#define PTP_SYNC_PKT_START	1
> +#define PTP_SYNC_PKT_PROBE	2
> +#define PTP_SYNC_PKT_PROBES	3
> +#define PTP_SYNC_PKT_OFFSET	4
> +#define PTP_SYNC_PKT_END	5
> +
> +/* print time sync debug messages */
> +#define TSYNC_DEBUG
> +
> +struct ptp_clock_sync {
> +	struct tep_handle	*tep;
> +	int			raw_id;
> +	int			marker_fd;
> +	int			series_id;
> +	int			flags;
> +	int			debug_fd;
> +};
> +
> +enum {
> +/* Consider only the probe with fastest response time,
> + * otherwise make a histogram from all probes.
> + */
> +	PTP_FLAG_FASTEST_RESPONSE		= (1 << 0),
> +/* Use trace marker to get the clock,
> + * otherwise use directly the system clock
> + */

Nit, multi-line comments should be:

/*
 * Use trace marker to get the clock,
 * otherwise use directly the system clock
 */

Although Linux networking code does the above, that's only allowed because
Linus likes David Miller, and he allows him to request that in his
networking code. ;-)

Also, the grammar of the above should be:

 "Use trace marker to get the clock, otherwise use the system clock
 directly."



> +	PTP_FLAG_USE_MARKER			= (1 << 1),
> +};
> +static int ptp_flags = PTP_FLAG_FASTEST_RESPONSE | PTP_FLAG_USE_MARKER;
> +
> +struct ptp_clock_start_msg {
> +	be32	series_id;
> +	be32	flags;
> +} __attribute__((packed));

We should define a macro: __packed and use that instead. Makes the code
look cleaner.

> +
> +struct ptp_clock_sample {
> +	s64		ts;
> +	be32		id;
> +} __attribute__((packed));
> +
> +struct ptp_clock_result_msg {
> +	be32			series_id;
> +	be32			count;
> +	struct ptp_clock_sample	samples[2*PTP_SYNC_LOOP];
> +} __attribute__((packed));
> +
> +struct ptp_clock_offset_msg {
> +	s64	ts;
> +	s64	offset;
> +};
> +
> +struct ptp_markers_context {
> +	struct clock_sync_context	*clock;
> +	struct ptp_clock_sync		*ptp;
> +	int				size;
> +	struct ptp_clock_result_msg	msg;
> +};

Me being picky about holes in structures, as "int" is 4 bytes, it should be
either at the end or mapped with other elements of 4 bytes. That is, msg
and size should be swapped.

> +
> +struct ptp_marker_buf {
> +	int local_cid;
> +	int remote_cid;
> +	int count;
> +	int packet_id;
> +} __attribute__((packed));
> +
> +struct ptp_marker {
> +	int series_id;
> +	struct ptp_marker_buf data;
> +} __attribute__((packed));
> +
> +static int ptp_clock_sync_init(struct tracecmd_time_sync *tsync)
> +{
> +	const char *systems[] = {"ftrace", NULL};
> +	struct clock_sync_context *clock_context;
> +	struct ptp_clock_sync *ptp;
> +	struct tep_event *raw;
> +	char *path;
> +
> +	if (!tsync || !tsync->context)
> +		return -1;
> +	clock_context = (struct clock_sync_context *)tsync->context;
> +	if (clock_context->proto_data)
> +		return 0;
> +
> +	ptp = calloc(1, sizeof(struct ptp_clock_sync));
> +	if (!ptp)
> +		return -1;
> +
> +	ptp->marker_fd = -1;
> +	ptp->debug_fd = -1;
> +
> +	path = tracefs_instance_get_dir(clock_context->instance);
> +	if (!path)
> +		goto error;
> +	ptp->tep = tracefs_local_events_system(path, systems);
> +	tracefs_put_tracing_file(path);
> +	if (!ptp->tep)
> +		goto error;
> +	raw = tep_find_event_by_name(ptp->tep, "ftrace", "raw_data");
> +	if (!raw)
> +		goto error;
> +
> +	ptp->raw_id = raw->id;
> +	tep_set_file_bigendian(ptp->tep, tracecmd_host_bigendian());
> +	tep_set_local_bigendian(ptp->tep, tracecmd_host_bigendian());
> +
> +	path = tracefs_instance_get_file(clock_context->instance, "trace_marker_raw");
> +	if (!path)
> +		goto error;
> +	ptp->marker_fd = open(path, O_WRONLY);
> +	tracefs_put_tracing_file(path);
> +
> +	clock_context->proto_data = ptp;
> +
> +#ifdef TSYNC_DEBUG
> +	if (clock_context->is_server) {
> +		char buff[256];
> +		int res_fd;
> +
> +		sprintf(buff, "res-cid%d.txt", clock_context->remote_cid);
> +
> +		res_fd = open(buff, O_CREAT|O_WRONLY|O_TRUNC, 0644);
> +		if (res_fd > 0)
> +			close(res_fd);
> +	}
> +#endif
> +
> +	return 0;
> +
> +error:
> +	if (ptp) {
> +		tep_free(ptp->tep);
> +		if (ptp->marker_fd >= 0)
> +			close(ptp->marker_fd);
> +	}
> +	free(ptp);
> +	return -1;
> +}
> +
> +static int ptp_clock_sync_free(struct tracecmd_time_sync *tsync)
> +{
> +	struct clock_sync_context *clock_context;
> +	struct ptp_clock_sync *ptp;
> +
> +	if (!tsync || !tsync->context)
> +		return -1;
> +	clock_context = (struct clock_sync_context *)tsync->context;
> +
> +	if (clock_context && clock_context->proto_data) {
> +		ptp = (struct ptp_clock_sync *)clock_context->proto_data;
> +		tep_free(ptp->tep);
> +		if (ptp->marker_fd >= 0)
> +			close(ptp->marker_fd);
> +		if (ptp->debug_fd >= 0)
> +			close(ptp->debug_fd);
> +		free(clock_context->proto_data);
> +		clock_context->proto_data = NULL;
> +	}
> +	return 0;
> +}
> +
> +static void ptp_probe_store(struct ptp_markers_context *ctx,
> +			    struct ptp_marker *marker,
> +			    unsigned long long ts)
> +{
> +	int index = -1;
> +
> +	if (marker->data.packet_id == 'r' &&
> +	    marker->data.count <= ctx->size) {
> +		index = marker->data.count - 1;
> +	} else if (marker->data.packet_id == 's' &&
> +		  marker->data.count*2 <= ctx->size){
> +		index = ctx->size / 2 + marker->data.count - 1;

These calculations should be turned into macros, or at the very least have
comments to why this is done.

Also, data.count for both should always be less than or equal ctx->size /
2, right?

If the ctx->size is for both, then the count should only be half. Wouldn't
the 'r' packet start writing over the 's' packets if it is not? If this is
the case, then we could simplify this to:

	if (marker->data.count > ctx->size / 2)
		return;

	index = marker->data_count - 1;

	switch (marker->data.packet_id) {
	case 'r':
		break;
	case 's':
		index += ctx->size / 2;
		break;
	default:
		return;
	}

	ctx->msg.samples[index].id = marker->data.count;
	ctx->msg.samples[index].ts = ts;
	ctx->msg.count++;

BTW, when would the samples[index].id ever equal to something other than
the index + 1, or index + size / 2 + 1?

> +	}
> +
> +	if (index >= 0) {
> +		ctx->msg.samples[index].id = marker->data.count;
> +		ctx->msg.samples[index].ts = ts;
> +		ctx->msg.count++;
> +	}
> +}
> +
> +static int ptp_marker_find(struct tep_event *event, struct tep_record *record,
> +			   int cpu, void *context)
> +{
> +	struct ptp_markers_context *ctx;
> +	struct tep_format_field *field;
> +	struct tep_format_field *id;
> +	struct ptp_marker *marker;
> +
> +	ctx = (struct ptp_markers_context *)context;
> +
> +	/* Make sure this is our event */
> +	if (event->id != ctx->ptp->raw_id)
> +		return 0;
> +	id = tep_find_field(event, "id");
> +	field = tep_find_field(event, "buf");

These probably could be saved and reused, instead of searching for them
each time. Could probably be saved in the context before even calling this.
Save the id and field when saving the raw_id.

> +	if (field && id &&
> +	    record->size >= (id->offset + sizeof(struct ptp_marker))) {
> +		marker = (struct ptp_marker *)(record->data + id->offset);
> +		if (marker->data.local_cid == ctx->clock->local_cid &&
> +		    marker->data.remote_cid == ctx->clock->remote_cid &&
> +		    marker->series_id == ctx->ptp->series_id &&
> +		    marker->data.count)
> +			ptp_probe_store(ctx, marker, record->ts);
> +	}
> +
> +	return 0;
> +}
> +
> +static int ptp_calc_offset_fastest(struct clock_sync_context *clock,
> +			   struct ptp_clock_result_msg *server,
> +			   struct ptp_clock_result_msg *client,
> +			   long long *offset_ret, long long *ts_ret,
> +			   int *bad_probes)
> +{
> +	struct ptp_clock_sample *sample_send;
> +	long long delta_min = 0;

If you set delta_min to LLONG_MAX...

> +	long long offset = 0;
> +	long long delta = 0;
> +	long long ts = 0;
> +	int i;
> +
> +	*bad_probes = 0;
> +	sample_send = server->samples + (server->count / 2);
> +	for (i = 0; i * 2 < server->count && i < client->count; i++) {

Instead of both checks, you could have:

	max_i = server->count / 2 < client->count ? server->count / 2 :
		client->count;

	for (i = 0; i < max_i; i++) {

> +		if (!server->samples[i].ts ||
> +		    !sample_send[i].ts ||
> +		    !client->samples[i].ts ||
> +		    server->samples[i].id != sample_send[i].id ||
> +		    server->samples[i].id != client->samples[i].id) {
> +			(*bad_probes)++;
> +			continue;
> +		}
> +
> +		ts = (sample_send[i].ts + server->samples[i].ts) / 2;
> +		offset = client->samples[i].ts - ts;
> +
> +		delta = server->samples[i].ts - sample_send[i].ts;
> +		if (!delta_min || delta_min > delta) {

... you can make this just (delta_min > delta) and not check the !delta_min.

> +			delta_min = delta;
> +			*offset_ret = offset;
> +			*ts_ret = ts;
> +		}
> +#ifdef TSYNC_DEBUG
> +		{
> +			struct ptp_clock_sync *ptp;
> +
> +			ptp = (struct ptp_clock_sync *)clock->proto_data;
> +			if (ptp && ptp->debug_fd > 0) {
> +				char buff[256];
> +
> +				sprintf(buff, "%lld %lld %lld\n",
> +					ts, client->samples[i].ts, offset);
> +				write(ptp->debug_fd, buff, strlen(buff));
> +			}
> +		}
> +#endif
> +	}
> +
> +	return 0;
> +}
> +
> +static int ptp_calc_offset_hist(struct clock_sync_context *clock,
> +			   struct ptp_clock_result_msg *server,
> +			   struct ptp_clock_result_msg *client,
> +			   long long *offset_ret, long long *ts_ret,
> +			   int *bad_probes)
> +{
> +	struct ptp_clock_sample *sample_send;
> +	long long timestamps[PTP_SYNC_LOOP];
> +	long long offsets[PTP_SYNC_LOOP];
> +	int hist[PTP_SYNC_LOOP];
> +	long long offset_max = 0;
> +	long long offset_min = 0;
> +	int ind, max = 0;
> +	long long bin;
> +	int i, k = 0;
> +
> +	*bad_probes = 0;
> +	memset(hist, 0, sizeof(int) * PTP_SYNC_LOOP);
> +	sample_send = server->samples + (server->count / 2);
> +	for (i = 0; i * 2 < server->count && i < client->count; i++) {
> +		if (!server->samples[i].ts ||
> +		    !sample_send[i].ts ||
> +		    !client->samples[i].ts ||
> +		    server->samples[i].id != sample_send[i].id ||
> +		    server->samples[i].id != client->samples[i].id) {
> +			(*bad_probes)++;
> +			continue;

Since this check is done twice, it would be good to make it into a static
inline function:

static inline bool good_probe(struct ptp_clock_sample *server_sample,
			      struct ptp_clock_sample *send_sample,
			      struct ptp_clock_sample *client_sample,
			      int *bad_probes)
{
	if (server_sample->ts && send_sample->ts && client_sample->ts &&
	    server_sample->id == send_sample->id &&
	    server_sample->id == client_sample->id)
		return true;
	(*bad_probes)++;
	return false;
}

Then you just need:

		if (!good_sample(&server->samples[i], &sample_send[i],
				 &client->samples[i], bad_probes))
			continue;

for both locations.

> +		}
> +
> +		timestamps[k] = (sample_send[i].ts + server->samples[i].ts) / 2;
> +		offsets[k] = client->samples[i].ts - timestamps[k];
> +		if (!offset_max || offset_max < llabs(offsets[k]))
> +			offset_max = llabs(offsets[k]);

No need for the offset_max check.

> +		if (!offset_min || offset_min > llabs(offsets[k]))

Same trick with LLONG_MAX.

> +			offset_min = llabs(offsets[k]);
> +#ifdef TSYNC_DEBUG
> +		{
> +			struct ptp_clock_sync *ptp;
> +
> +			ptp = (struct ptp_clock_sync *)clock->proto_data;
> +
> +			if (ptp && ptp->debug_fd > 0) {
> +				char buff[256];
> +
> +				sprintf(buff, "%lld %lld %lld\n",
> +					timestamps[k],
> +					client->samples[i].ts, offsets[k]);
> +				write(ptp->debug_fd, buff, strlen(buff));
> +			}
> +		}
> +#endif
> +		k++;
> +	}
> +
> +	bin = (offset_max - offset_min) / PTP_SYNC_LOOP;
> +	for (i = 0; i < k; i++) {
> +		ind = (llabs(offsets[i]) - offset_min) / bin;
> +		if (ind < PTP_SYNC_LOOP) {
> +			hist[ind]++;
> +			if (max < hist[ind]) {
> +				max = hist[ind];
> +				*offset_ret = offsets[i];
> +				*ts_ret = timestamps[i];
> +			}

I'm curious to how accurate this is?

-- Steve

> +		}
> +	}
> +
> +	return 0;
> +}
> +

^ permalink raw reply	[flat|nested] 16+ messages in thread

* Re: [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name
  2020-10-09 14:03 ` [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name Tzvetomir Stoyanov (VMware)
@ 2020-10-22  1:26   ` Steven Rostedt
  2020-10-22  1:31     ` Steven Rostedt
  0 siblings, 1 reply; 16+ messages in thread
From: Steven Rostedt @ 2020-10-22  1:26 UTC (permalink / raw)
  To: Tzvetomir Stoyanov (VMware); +Cc: linux-trace-devel

On Fri,  9 Oct 2020 17:03:31 +0300
"Tzvetomir Stoyanov (VMware)" <tz.stoyanov@gmail.com> wrote:

> Added enum with ftrace clock IDs and APIs to convert ftrace name to ID
> and vice versa, as part of libtracecmd.
> 
> Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
> ---
>  include/trace-cmd/trace-cmd.h | 16 +++++++++++++
>  lib/trace-cmd/trace-util.c    | 45 +++++++++++++++++++++++++++++++++++
>  2 files changed, 61 insertions(+)
> 
> diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
> index f9c1f843..393a2e7b 100644
> --- a/include/trace-cmd/trace-cmd.h
> +++ b/include/trace-cmd/trace-cmd.h
> @@ -415,6 +415,22 @@ int tracecmd_msg_recv_time_sync(struct tracecmd_msg_handle *msg_handle,
>  				unsigned int *sync_msg_id,
>  				unsigned int *payload_size, char **payload);
>  
> +enum tracecmd_clocks {
> +	TRACECMD_CLOCK_UNKNOWN	= 0,
> +	TRACECMD_CLOCK_LOCAL	= 1,
> +	TRACECMD_CLOCK_GLOBAL	= 1 << 1,
> +	TRACECMD_CLOCK_COUNTER	= 1 << 2,
> +	TRACECMD_CLOCK_UPTIME	= 1 << 3,
> +	TRACECMD_CLOCK_PERF	= 1 << 4,
> +	TRACECMD_CLOCK_MONO	= 1 << 5,
> +	TRACECMD_CLOCK_MONO_RAW	= 1 << 6,
> +	TRACECMD_CLOCK_BOOT	= 1 << 7,
> +	TRACECMD_CLOCK_X86_TSC	= 1 << 8

I'm curious to why you have this as a bitmask. We can only have on
clock at a time, right?

Also, if we make it a simple counter, we can create a string as well:

#define TRACECMD_CLOCKS \
 C(UNKNOWN,	unknown),	\
 C(LOCAL,	local),	\
 C(GLOBAL,	global),\
 C(COUNTER,	counter),\
 C(UPTIME,	uptime),\
 C(PERF,	perf),	\
 C(MONO,	mono),	\
 C(MONO_RAW,	mono_raw),\
 C(BOOT,	boot,	\
 C(X86_TSC,	x86-tsc)

#undef C
#define C(a, b) TRACECMD_CLOCK_##a

enum tracecmd_clocks { TRACECMD_CLOCKS };

#undef C

> +};
> +
> +enum tracecmd_clocks tracecmd_clock_str2id(const char *clock);
> +char *tracecmd_clock_id2str(enum tracecmd_clocks clock);
> +
>  /* --- Timestamp synchronization --- */
>  
>  enum{
> diff --git a/lib/trace-cmd/trace-util.c b/lib/trace-cmd/trace-util.c
> index 0ead96ea..e20362e3 100644
> --- a/lib/trace-cmd/trace-util.c
> +++ b/lib/trace-cmd/trace-util.c
> @@ -33,6 +33,51 @@ static bool debug;
>  
>  static FILE *logfp;
>  
> +const static struct {
> +	char *clock_str;
> +	enum tracecmd_clocks clock_id;
> +} trace_clocks[] = {
> +	{"local", TRACECMD_CLOCK_LOCAL},
> +	{"global", TRACECMD_CLOCK_GLOBAL},
> +	{"counter", TRACECMD_CLOCK_COUNTER},
> +	{"uptime", TRACECMD_CLOCK_UPTIME},
> +	{"perf", TRACECMD_CLOCK_PERF},
> +	{"mono", TRACECMD_CLOCK_MONO},
> +	{"mono_raw", TRACECMD_CLOCK_MONO_RAW},
> +	{"boot", TRACECMD_CLOCK_BOOT},
> +	{"x86-tsc", TRACECMD_CLOCK_X86_TSC},
> +	{NULL, -1}
> +};

He we would have:

#define C(a, b) #b
const char * trace_clocks[] = { TRACECMD_CLOCKS }

> +
> +/**
> + * tracecmd_clock_str2id - Convert ftrace clock name to clock ID
> + * @clock: Ftrace clock name
> + * Returns ID of the ftrace clock
> + */
> +enum tracecmd_clocks tracecmd_clock_str2id(const char *clock)
> +{
	int i;

btw, in normal C, it's not recommended to declare a counter in a loop.

	for (i = 0; i < ARRAY_SIZE(trace_clocks); i++) {
		if (strcmp(trace_clocks[i], clock) == 0)
			return i;
	return 0;
}


> +	for (int i = 0; trace_clocks[i].clock_str; i++) {
> +		if (!strncmp(clock, trace_clocks[i].clock_str,
> +		    strlen(trace_clocks[i].clock_str)))
> +			return trace_clocks[i].clock_id;
> +	}
> +	return TRACECMD_CLOCK_UNKNOWN;
> +}
> +
> +/**
> + * tracecmd_clock_id2str - Convert clock ID to ftare clock name
> + * @clock: Clock ID
> + * Returns name of a ftrace clock
> + */
> +char *tracecmd_clock_id2str(enum tracecmd_clocks clock)

Should probably have this return const char *.

Such that callers wont modify it.

> +{

And here we would have:

	if (clock < ARRAY_SIZE(trace_clocks))
		return trace_clocks[i];

	return NULL;

-- Steve

> +	for (int i = 0; trace_clocks[i].clock_str; i++) {
> +		if (trace_clocks[i].clock_id == clock)
> +			return trace_clocks[i].clock_str;
> +	}
> +	return NULL;
> +}
> +
>  /**
>   * tracecmd_set_debug - Set debug mode of the tracecmd library
>   * @set_debug: The new "debug" mode. If true, the tracecmd library is


^ permalink raw reply	[flat|nested] 16+ messages in thread

* Re: [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name
  2020-10-22  1:26   ` Steven Rostedt
@ 2020-10-22  1:31     ` Steven Rostedt
  0 siblings, 0 replies; 16+ messages in thread
From: Steven Rostedt @ 2020-10-22  1:31 UTC (permalink / raw)
  To: Tzvetomir Stoyanov (VMware); +Cc: linux-trace-devel

On Wed, 21 Oct 2020 21:26:26 -0400
Steven Rostedt <rostedt@goodmis.org> wrote:

> On Fri,  9 Oct 2020 17:03:31 +0300
> "Tzvetomir Stoyanov (VMware)" <tz.stoyanov@gmail.com> wrote:
> 
> > Added enum with ftrace clock IDs and APIs to convert ftrace name to ID
> > and vice versa, as part of libtracecmd.
> > 
> > Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
> > ---
> >  include/trace-cmd/trace-cmd.h | 16 +++++++++++++
> >  lib/trace-cmd/trace-util.c    | 45 +++++++++++++++++++++++++++++++++++
> >  2 files changed, 61 insertions(+)
> > 
> > diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
> > index f9c1f843..393a2e7b 100644
> > --- a/include/trace-cmd/trace-cmd.h
> > +++ b/include/trace-cmd/trace-cmd.h
> > @@ -415,6 +415,22 @@ int tracecmd_msg_recv_time_sync(struct tracecmd_msg_handle *msg_handle,
> >  				unsigned int *sync_msg_id,
> >  				unsigned int *payload_size, char **payload);
> >  
> > +enum tracecmd_clocks {
> > +	TRACECMD_CLOCK_UNKNOWN	= 0,
> > +	TRACECMD_CLOCK_LOCAL	= 1,
> > +	TRACECMD_CLOCK_GLOBAL	= 1 << 1,
> > +	TRACECMD_CLOCK_COUNTER	= 1 << 2,
> > +	TRACECMD_CLOCK_UPTIME	= 1 << 3,
> > +	TRACECMD_CLOCK_PERF	= 1 << 4,
> > +	TRACECMD_CLOCK_MONO	= 1 << 5,
> > +	TRACECMD_CLOCK_MONO_RAW	= 1 << 6,
> > +	TRACECMD_CLOCK_BOOT	= 1 << 7,
> > +	TRACECMD_CLOCK_X86_TSC	= 1 << 8  
> 
> I'm curious to why you have this as a bitmask. We can only have on
> clock at a time, right?

I got to patch 5 and see that you do need this to be a bitmask.

When this is the case, the change log should state that. That is, the
change log should have something like:

The clock enum will be used in a bitmask such that the synchronization
protocol can pass a bitmask of supported clocks.

Remember, all patches should be "stand alone". That is, do not assume
that someone will have access to other patches when they are looking at
the current patch.

You may disregard the rest of this email.

-- Steve

^ permalink raw reply	[flat|nested] 16+ messages in thread

* Re: [PATCH v24 05/10] trace-cmd: Add clock parameter to timestamp synchronization plugins
  2020-10-09 14:03 ` [PATCH v24 05/10] trace-cmd: Add clock parameter to timestamp synchronization plugins Tzvetomir Stoyanov (VMware)
@ 2020-10-22  1:39   ` Steven Rostedt
  0 siblings, 0 replies; 16+ messages in thread
From: Steven Rostedt @ 2020-10-22  1:39 UTC (permalink / raw)
  To: Tzvetomir Stoyanov (VMware); +Cc: linux-trace-devel

On Fri,  9 Oct 2020 17:03:33 +0300
"Tzvetomir Stoyanov (VMware)" <tz.stoyanov@gmail.com> wrote:

> Some timestamp synchronization plugins may not support all ftrace
> clocks. Added logic to timestamp synchronization plugins to declare what
> ftace clocks they support. Added logic to select plugin depending on the

 s/ftace/ftrace/

> ftrace clock used in the current trace session and supported clocks.
> 
> Signed-off-by: Tzvetomir Stoyanov (VMware) <tz.stoyanov@gmail.com>
> ---
>  include/trace-cmd/trace-cmd.h             |  4 ++--
>  lib/trace-cmd/include/trace-tsync-local.h |  1 +
>  lib/trace-cmd/trace-timesync-ptp.c        |  1 +
>  lib/trace-cmd/trace-timesync.c            | 21 +++++++++++++++++++--
>  tracecmd/trace-record.c                   |  4 +++-
>  tracecmd/trace-tsync.c                    |  2 +-
>  6 files changed, 27 insertions(+), 6 deletions(-)
> 
> diff --git a/include/trace-cmd/trace-cmd.h b/include/trace-cmd/trace-cmd.h
> index 393a2e7b..66b5d02b 100644
> --- a/include/trace-cmd/trace-cmd.h
> +++ b/include/trace-cmd/trace-cmd.h
> @@ -455,8 +455,8 @@ struct tracecmd_time_sync {
>  };
>  
>  void tracecmd_tsync_init(void);
> -int tracecmd_tsync_proto_getall(char **proto_mask, int *words);
> -unsigned int tracecmd_tsync_proto_select(char *proto_mask, int words);
> +int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock);
> +unsigned int tracecmd_tsync_proto_select(char *proto_mask, int words, char *clock);
>  bool tsync_proto_is_supported(unsigned int proto_id);
>  void tracecmd_tsync_with_host(struct tracecmd_time_sync *tsync);
>  void tracecmd_tsync_with_guest(struct tracecmd_time_sync *tsync);
> diff --git a/lib/trace-cmd/include/trace-tsync-local.h b/lib/trace-cmd/include/trace-tsync-local.h
> index 1de9d5e5..37c3597b 100644
> --- a/lib/trace-cmd/include/trace-tsync-local.h
> +++ b/lib/trace-cmd/include/trace-tsync-local.h
> @@ -27,6 +27,7 @@ struct clock_sync_context {
>  };
>  
>  int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
> +				  int supported_clocks,
>  				int (*init)(struct tracecmd_time_sync *),
>  				int (*free)(struct tracecmd_time_sync *),
>  				int (*calc)(struct tracecmd_time_sync *,
> diff --git a/lib/trace-cmd/trace-timesync-ptp.c b/lib/trace-cmd/trace-timesync-ptp.c
> index 19e2f647..51f17bca 100644
> --- a/lib/trace-cmd/trace-timesync-ptp.c
> +++ b/lib/trace-cmd/trace-timesync-ptp.c
> @@ -678,6 +678,7 @@ int ptp_clock_sync_register(void)
>  {
>  	return tracecmd_tsync_proto_register(TRACECMD_TIME_SYNC_PROTO_PTP,
>  					     TRACECMD_TIME_SYNC_PROTO_PTP_WEIGHT,
> +					     0,
>  					     ptp_clock_sync_init,
>  					     ptp_clock_sync_free,
>  					     ptp_clock_sync_calc);
> diff --git a/lib/trace-cmd/trace-timesync.c b/lib/trace-cmd/trace-timesync.c
> index e294698b..47d34e2a 100644
> --- a/lib/trace-cmd/trace-timesync.c
> +++ b/lib/trace-cmd/trace-timesync.c
> @@ -26,6 +26,7 @@ struct tsync_proto {
>  	struct tsync_proto *next;
>  	unsigned int proto_id;
>  	int	weight;
> +	int supported_clocks;
>  
>  	int (*clock_sync_init)(struct tracecmd_time_sync *clock_context);
>  	int (*clock_sync_free)(struct tracecmd_time_sync *clock_context);
> @@ -55,6 +56,7 @@ void tracecmd_tsync_init(void)
>  }
>  
>  int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
> +				int supported_clocks,
>  				int (*init)(struct tracecmd_time_sync *),
>  				int (*free)(struct tracecmd_time_sync *),
>  				int (*calc)(struct tracecmd_time_sync *,
> @@ -69,6 +71,7 @@ int tracecmd_tsync_proto_register(unsigned int proto_id, int weight,
>  		return -1;
>  	proto->proto_id = proto_id;
>  	proto->weight = weight;
> +	proto->supported_clocks = supported_clocks;
>  	proto->clock_sync_init = init;
>  	proto->clock_sync_free = free;
>  	proto->clock_sync_calc = calc;
> @@ -139,19 +142,26 @@ int tracecmd_tsync_get_offsets(struct tracecmd_time_sync *tsync,
>   *
>   * @proto_mask: bitmask array of time sync protocols, supported by the peer
>   * @length: size of the @protos array
> + * @clock: selected trace clock
>   *
>   * Retuns Id of a time sync protocol, that can be used with the peer, or 0
>   *	  in case there is no match with supported protocols
>   */
> -unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length)
> +unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length, char *clock)
>  {
>  	struct tsync_proto *selected = NULL;
>  	struct tsync_proto *proto;
> +	int clock_id = 0;
>  	int word;
>  	int id;
>  
> +	if (clock)
> +		clock_id = tracecmd_clock_str2id(clock);

I wonder if we should just have tracecmd_clock_str2id(NULL) return zero.

Then you don't need the 'if (clock)' check.


>  	for (word = 0; word < length; word++) {
>  		for (proto = tsync_proto_list; proto; proto = proto->next) {
> +			if (proto->supported_clocks && clock_id &&
> +			    !(proto->supported_clocks & clock_id))
> +				continue;
>  			if (proto->proto_id < word * PROTO_MASK_SIZE)
>  				continue;
>  
> @@ -181,18 +191,22 @@ unsigned int tracecmd_tsync_proto_select(char *proto_mask, int length)
>   * @proto_mask: return, allocated bitmask array of time sync protocols,
>   *	       supported by the peer. Must be freed by free()
>   * @words: return, allocated size of the @protobits array
> + * @clock: selected trace clock
>   *
>   * If completed successfully 0 is returned and allocated array in @proto_mask of
>   * size @words. In case of an error, -1 is returned.
>   * @proto_mask must be freed with free()
>   */
> -int tracecmd_tsync_proto_getall(char **proto_mask, int *words)
> +int tracecmd_tsync_proto_getall(char **proto_mask, int *words, const char *clock)
>  {
>  	struct tsync_proto *proto;
>  	int proto_max = 0;
> +	int clock_id = 0;
>  	int count = 0;
>  	char *protos;
>  
> +	if (clock)
> +		clock_id =  tracecmd_clock_str2id(clock);
>  	for (proto = tsync_proto_list; proto; proto = proto->next)
>  		if (proto->proto_id > proto_max)
>  			proto_max = proto->proto_id;
> @@ -205,6 +219,9 @@ int tracecmd_tsync_proto_getall(char **proto_mask, int *words)
>  	for (proto = tsync_proto_list; proto; proto = proto->next) {
>  		if ((proto->proto_id / PROTO_MASK_SIZE) >= count)
>  			continue;
> +		if (proto->supported_clocks && clock_id &&
> +		    !(proto->supported_clocks & clock_id))
> +			continue;
>  		protos[proto->proto_id / PROTO_MASK_SIZE] |=
>  				(1 << (proto->proto_id % PROTO_MASK_SIZE));
>  	}
> diff --git a/tracecmd/trace-record.c b/tracecmd/trace-record.c
> index 9149cf76..08bbc0e0 100644
> --- a/tracecmd/trace-record.c
> +++ b/tracecmd/trace-record.c
> @@ -3677,7 +3677,7 @@ static void connect_to_agent(struct buffer_instance *instance)
>  		die("Failed to allocate message handle");
>  
>  	if (instance->tsync.loop_interval >= 0)
> -		tracecmd_tsync_proto_getall(&protos, &protos_count);
> +		tracecmd_tsync_proto_getall(&protos, &protos_count, instance->clock);
>  
>  	ret = tracecmd_msg_send_trace_req(msg_handle, instance->argc,
>  					  instance->argv, use_fifos,
> @@ -6111,6 +6111,8 @@ static void parse_record_options(int argc,
>  						 (char *)top_instance.clock,
>  						 true);
>  					add_argv(instance, "-C", true);
> +					if (!instance->clock)
> +						instance->clock = strdup((char *)top_instance.clock);

Need to check the success of the allocation.

-- Steve

>  				}
>  			}
>  			instance->tsync.loop_interval = top_instance.tsync.loop_interval;
> diff --git a/tracecmd/trace-tsync.c b/tracecmd/trace-tsync.c
> index e639788d..832b70c3 100644
> --- a/tracecmd/trace-tsync.c
> +++ b/tracecmd/trace-tsync.c
> @@ -220,7 +220,7 @@ unsigned int tracecmd_guest_tsync(char *tsync_protos,
>  	int fd;
>  
>  	fd = -1;
> -	proto = tracecmd_tsync_proto_select(tsync_protos, tsync_protos_size);
> +	proto = tracecmd_tsync_proto_select(tsync_protos, tsync_protos_size, clock);
>  	if (!proto)
>  		return 0;
>  #ifdef VSOCK


^ permalink raw reply	[flat|nested] 16+ messages in thread

* Re: [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization
  2020-10-15 21:24   ` Steven Rostedt
@ 2020-10-26  7:38     ` Tzvetomir Stoyanov
  0 siblings, 0 replies; 16+ messages in thread
From: Tzvetomir Stoyanov @ 2020-10-26  7:38 UTC (permalink / raw)
  To: Steven Rostedt; +Cc: Linux Trace Devel

On Fri, Oct 16, 2020 at 12:24 AM Steven Rostedt <rostedt@goodmis.org> wrote:
[...]
> > +
> > +static void ptp_probe_store(struct ptp_markers_context *ctx,
> > +                         struct ptp_marker *marker,
> > +                         unsigned long long ts)
> > +{
> > +     int index = -1;
> > +
> > +     if (marker->data.packet_id == 'r' &&
> > +         marker->data.count <= ctx->size) {
> > +             index = marker->data.count - 1;
> > +     } else if (marker->data.packet_id == 's' &&
> > +               marker->data.count*2 <= ctx->size){
> > +             index = ctx->size / 2 + marker->data.count - 1;
>
> These calculations should be turned into macros, or at the very least have
> comments to why this is done.
>
> Also, data.count for both should always be less than or equal ctx->size /
> 2, right?
>
> If the ctx->size is for both, then the count should only be half. Wouldn't
> the 'r' packet start writing over the 's' packets if it is not? If this is
> the case, then we could simplify this to:
>
>         if (marker->data.count > ctx->size / 2)
>                 return;
>
>         index = marker->data_count - 1;
>
>         switch (marker->data.packet_id) {
>         case 'r':
>                 break;
>         case 's':
>                 index += ctx->size / 2;
>                 break;
>         default:
>                 return;
>         }
>
>         ctx->msg.samples[index].id = marker->data.count;
>         ctx->msg.samples[index].ts = ts;
>         ctx->msg.count++;
>
> BTW, when would the samples[index].id ever equal to something other than
> the index + 1, or index + size / 2 + 1?
>
I'll add comment to the ptp_probe_store() function, describing its
logic. It is a little
bit confusing, as it handles both cases - server and client. The
server tracks both sent
and returned probes, in that case the array size is 2 * max data count
- first are stored
returned probes, after them are sent probes. In the client context,
there are only returned
probes and the array size is max data count. The samples[index].id is
always the count
of the current probe, it could be index + 1 or index + size / 2 + 1
only, no other values.
I store it in the ID field, as in the server context there are two
entries (sent and received)
with the same probe count, and it is easier to match both based on
this ID when do the
calculations. I can use the index only, to find both sent and returned
probes and to assume
that both probes match, but I prefered to use this ID to verify if
both records are from
the same probe.

[...]
> > +
> > +     bin = (offset_max - offset_min) / PTP_SYNC_LOOP;
> > +     for (i = 0; i < k; i++) {
> > +             ind = (llabs(offsets[i]) - offset_min) / bin;
> > +             if (ind < PTP_SYNC_LOOP) {
> > +                     hist[ind]++;
> > +                     if (max < hist[ind]) {
> > +                             max = hist[ind];
> > +                             *offset_ret = offsets[i];
> > +                             *ts_ret = timestamps[i];
> > +                     }
>
> I'm curious to how accurate this is?
>
Using the histogram logic, an accuracy from 200ns up to 15 000ns can
be achieved,
results vary a lot. The accuracy of the fastest response logic is
around 1000ns - 3000ns usually.

> -- Steve

Thanks Steve!
I'll send the next version of the patch set addressing these comments.

>
> > +             }
> > +     }
> > +
> > +     return 0;
> > +}
> > +



--
Tzvetomir (Ceco) Stoyanov
VMware Open Source Technology Center

^ permalink raw reply	[flat|nested] 16+ messages in thread

end of thread, other threads:[~2020-10-26  7:38 UTC | newest]

Thread overview: 16+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-10-09 14:03 [PATCH v24 00/10] Timestamp synchronization of host - guest tracing session Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 01/10] trace-cmd: [POC] PTP-like algorithm for host - guest timestamp synchronization Tzvetomir Stoyanov (VMware)
2020-10-15 21:24   ` Steven Rostedt
2020-10-26  7:38     ` Tzvetomir Stoyanov
2020-10-09 14:03 ` [PATCH v24 02/10] trace-cmd: Debug scripts for " Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 03/10] trace-cmd: Add trace-cmd library APIs for ftrace clock name Tzvetomir Stoyanov (VMware)
2020-10-22  1:26   ` Steven Rostedt
2020-10-22  1:31     ` Steven Rostedt
2020-10-09 14:03 ` [PATCH v24 04/10] trace-cmd: Move VM related logic in a separate file Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 05/10] trace-cmd: Add clock parameter to timestamp synchronization plugins Tzvetomir Stoyanov (VMware)
2020-10-22  1:39   ` Steven Rostedt
2020-10-09 14:03 ` [PATCH v24 06/10] trace-cmd: Add role " Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 07/10] trace-cmd: Add host / guest role in timestamp synchronization context Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 08/10] trace-cmd: Add guest CPU count PID in tracecmd_time_sync struct Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 09/10] trace-cmd: Fixed bitmask logic tracecmd_tsync_proto_getall() Tzvetomir Stoyanov (VMware)
2020-10-09 14:03 ` [PATCH v24 10/10] trace-cmd [POC]: Add KVM timestamp synchronization plugin Tzvetomir Stoyanov (VMware)

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.