qemu-devel.nongnu.org archive mirror
 help / color / mirror / Atom feed
* [Qemu-devel] virtio-serial: An interface for host-guest communication
@ 2009-07-27 18:04 Amit Shah
  2009-07-27 18:04 ` [Qemu-devel] [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Amit Shah
  2009-07-27 20:22 ` [Qemu-devel] Re: virtio-serial: An interface for host-guest communication Anthony Liguori
  0 siblings, 2 replies; 62+ messages in thread
From: Amit Shah @ 2009-07-27 18:04 UTC (permalink / raw)
  To: virtualization; +Cc: qemu-devel, kvm


Hello all,

This are the latest version of the patches.

Lots of things have changed since the last submission. A few of
which I remember:
- VNC copy / paste works* (* conditions apply)
  - client vnc copies get propagated to guest port 3 (/dev/vmch3)
  - guest writes to port 3 (/dev/vmch3) go straight to client's clipboard
- sysfs hooks to autodiscover ports
- support for 64 ports in this version (MAX_VIRTIO_SERIAL_PORTS). More
  ports can be added by introducing a new feature flag to maintain
  backward compat. However, till this code gets accepted upstream, the
  value of that #define can change. I think 64 ports are enough for
  everyone.
- remove support for control queue (though this queue could make a
  comeback for just one use-case that I can currently think of:
  to prevent rogue (host) userspace putting lots of data into a guest
  that goes unconsumed for a while, increasing the memory pressure. To
  prevent this a threshold level can be decided upon and a control
  message can be sent to host userspace to prevent any more writes
  to the port.
- numerous fixes

There still exist a few kmalloc/kfree-related debug logs that spew up
in the guest but I haven't been able to track them down.

As for the merge with virtio-console, Christian has voiced some oppostion
to that idea. For the merge to happen, the kernel folks have to agree
in merging the driver as well and I can proceed once we have a resolution
on this.

Other than that, a few more rebases have to be done to the qemu code to
make it apply to qemu-upstream.

Please give this a good review. 

Thanks,
	Amit.

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

* [Qemu-devel] [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication
  2009-07-27 18:04 [Qemu-devel] virtio-serial: An interface for host-guest communication Amit Shah
@ 2009-07-27 18:04 ` Amit Shah
  2009-07-27 18:04   ` [Qemu-devel] [PATCH 1/3] virtio-serial: virtio device for simple host <-> guest communication Amit Shah
  2009-08-05  0:03   ` [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Rusty Russell
  2009-07-27 20:22 ` [Qemu-devel] Re: virtio-serial: An interface for host-guest communication Anthony Liguori
  1 sibling, 2 replies; 62+ messages in thread
From: Amit Shah @ 2009-07-27 18:04 UTC (permalink / raw)
  To: virtualization; +Cc: Amit Shah, qemu-devel, kvm

We expose multiple char devices ("ports") for simple communication
between the host userspace and guest.

Sample offline usages can be: poking around in a guest to find out
the file systems used, applications installed, etc. Online usages
can be sharing of clipboard data between the guest and the host,
sending information about logged-in users to the host, locking the
screen or session when a vnc session is closed, and so on.

The interface presented to guest userspace is of a simple char
device, so it can be used like this:

    fd = open("/dev/vmch0", O_RDWR);
    ret = read(fd, buf, 100);
    ret = write(fd, string, strlen(string));

Each port is to be assigned a unique function, for example, the first
4 ports may be reserved for libvirt usage, the next 4 for generic
streaming data and so on. This port-function mapping isn't finalised
yet.

For requirements, use-cases and some history see

    http://www.linux-kvm.org/page/VMchannel_Requirements

Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
 drivers/char/Kconfig          |    6 +
 drivers/char/Makefile         |    1 +
 drivers/char/virtio_serial.c  |  690 +++++++++++++++++++++++++++++++++++++++++
 include/linux/virtio_serial.h |   27 ++
 4 files changed, 724 insertions(+), 0 deletions(-)
 create mode 100644 drivers/char/virtio_serial.c
 create mode 100644 include/linux/virtio_serial.h

diff --git a/drivers/char/Kconfig b/drivers/char/Kconfig
index 6a06913..51adb48 100644
--- a/drivers/char/Kconfig
+++ b/drivers/char/Kconfig
@@ -679,6 +679,12 @@ config VIRTIO_CONSOLE
 	help
 	  Virtio console for use with lguest and other hypervisors.
 
+config VIRTIO_SERIAL
+	tristate "Virtio serial"
+	select VIRTIO
+	select VIRTIO_RING
+	help
+	  Virtio serial device driver for simple guest and host communication
 
 config HVCS
 	tristate "IBM Hypervisor Virtual Console Server support"
diff --git a/drivers/char/Makefile b/drivers/char/Makefile
index 66f779a..5e1915b 100644
--- a/drivers/char/Makefile
+++ b/drivers/char/Makefile
@@ -54,6 +54,7 @@ obj-$(CONFIG_HVC_XEN)		+= hvc_xen.o
 obj-$(CONFIG_HVC_IUCV)		+= hvc_iucv.o
 obj-$(CONFIG_HVC_UDBG)		+= hvc_udbg.o
 obj-$(CONFIG_VIRTIO_CONSOLE)	+= virtio_console.o
+obj-$(CONFIG_VIRTIO_SERIAL)	+= virtio_serial.o
 obj-$(CONFIG_RAW_DRIVER)	+= raw.o
 obj-$(CONFIG_SGI_SNSC)		+= snsc.o snsc_event.o
 obj-$(CONFIG_MSPEC)		+= mspec.o
diff --git a/drivers/char/virtio_serial.c b/drivers/char/virtio_serial.c
new file mode 100644
index 0000000..f42709d
--- /dev/null
+++ b/drivers/char/virtio_serial.c
@@ -0,0 +1,690 @@
+/*
+ * VirtIO Serial driver
+ *
+ * This is paravirtualised serial guest<->host communication channel
+ * for relaying short messages and events in either direction.
+ *
+ * One PCI device can have multiple serial ports so that different
+ * applications can communicate without polluting the PCI device space
+ * in the guest.
+ *
+ * Copyright (C) 2009, Red Hat, Inc.
+ *
+ * Author(s): Amit Shah <amit.shah@redhat.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA  02110-1301  USA
+ */
+
+#include <linux/cdev.h>
+#include <linux/completion.h>
+#include <linux/device.h>
+#include <linux/err.h>
+#include <linux/fs.h>
+#include <linux/init.h>
+#include <linux/poll.h>
+#include <linux/virtio.h>
+#include <linux/virtio_serial.h>
+#include <linux/workqueue.h>
+
+struct virtio_serial_struct {
+	struct work_struct rx_work;
+	struct work_struct tx_work;
+	struct work_struct queue_work;
+	struct work_struct config_work;
+
+	struct list_head port_head;
+
+	struct virtio_device *vdev;
+	struct class *class;
+	struct virtqueue *in_vq, *out_vq;
+
+	struct virtio_serial_config *config;
+};
+
+/* This struct holds individual buffers received for each port */
+struct virtio_serial_port_buffer {
+	struct list_head list;
+
+	unsigned int len; /* length of the buffer */
+	unsigned int offset; /* offset in the buf from which to consume data */
+
+	char *buf;
+};
+
+/* This struct is put in each buffer that gets passed to userspace and
+ * vice-versa
+ */
+struct virtio_serial_id {
+	u32 id; /* Port number */
+};
+
+struct virtio_serial_port {
+	/* Next port in the list */
+	struct list_head next;
+
+	/* Buffer management */
+	struct virtio_serial_port_buffer read_buf;
+	struct list_head readbuf_head;
+	struct completion have_data;
+
+	/* Each port associates with a separate char device */
+	struct cdev cdev;
+	struct device *dev;
+};
+
+static struct virtio_serial_struct virtserial;
+
+static int major = 60; /* from the experimental range */
+
+static struct virtio_serial_port *get_port_from_id(u32 id)
+{
+	struct virtio_serial_port *port;
+	struct list_head *ptr;
+
+	list_for_each(ptr, &virtserial.port_head) {
+		port = list_entry(ptr, struct virtio_serial_port, next);
+
+		if (MINOR(port->dev->devt) == id)
+			return port;
+	}
+	return NULL;
+}
+
+static int get_id_from_port(struct virtio_serial_port *port)
+{
+	struct virtio_serial_port *match;
+	struct list_head *ptr;
+
+	list_for_each(ptr, &virtserial.port_head) {
+		match = list_entry(ptr, struct virtio_serial_port, next);
+
+		if (match == port)
+			return MINOR(port->dev->devt);
+	}
+	return VIRTIO_SERIAL_BAD_ID;
+}
+
+static struct virtio_serial_port *get_port_from_buf(char *buf)
+{
+	u32 id;
+
+	memcpy(&id, buf, sizeof(id));
+
+	return get_port_from_id(id);
+}
+
+
+static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
+			       size_t count, loff_t *offp)
+{
+	struct list_head *ptr, *ptr2;
+	struct virtio_serial_port *port;
+	struct virtio_serial_port_buffer *buf;
+	ssize_t ret;
+
+	port = filp->private_data;
+
+	ret = -EINTR;
+	if (list_empty(&port->readbuf_head)) {
+		if (filp->f_flags & O_NONBLOCK)
+			return -EAGAIN;
+
+		if (wait_for_completion_interruptible(&port->have_data) < 0)
+			return ret;
+	}
+	list_for_each_safe(ptr, ptr2, &port->readbuf_head) {
+		buf = list_entry(ptr, struct virtio_serial_port_buffer, list);
+
+		/* FIXME: other buffers further in this list might
+		 * have data too
+		 */
+		if (count > buf->len - buf->offset)
+			count = buf->len - buf->offset;
+
+		ret = copy_to_user(ubuf, buf->buf + buf->offset, count);
+
+		/* Return the number of bytes actually copied */
+		ret = count - ret;
+
+		buf->offset += ret;
+
+		if (buf->len - buf->offset == 0) {
+			list_del(&buf->list);
+			kfree(buf->buf);
+			kfree(buf);
+		}
+		/* FIXME: if there's more data requested and more data
+		 * available, return it.
+		 */
+		break;
+	}
+	return ret;
+}
+
+/* For data that exceeds PAGE_SIZE in size we should send it all in
+ * one sg to not unnecessarily split up the data. Also some (all?)
+ * vnc clients don't consume split data.
+ *
+ * If we are to keep PAGE_SIZE sized buffers, we then have to stack
+ * multiple of those in one virtio request. virtio-ring returns to us
+ * just one pointer for all the buffers. So use this struct to
+ * allocate the bufs in so that freeing this up later is easier.
+ */
+struct vbuf {
+	char **bufs;
+	struct scatterlist *sg;
+	unsigned int nent;
+};
+
+static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
+				size_t count, loff_t *offp)
+{
+	struct virtqueue *out_vq;
+	struct virtio_serial_port *port;
+	struct virtio_serial_id id;
+	struct vbuf *vbuf;
+	size_t offset, size;
+	ssize_t ret;
+	int i, id_len;
+
+	port = filp->private_data;
+	id.id = get_id_from_port(port);
+	out_vq = virtserial.out_vq;
+
+	id_len = sizeof(id);
+
+	ret = -EFBIG;
+	vbuf = kzalloc(sizeof(struct vbuf), GFP_KERNEL);
+	if (!vbuf)
+		return ret;
+
+	/* Max. number of buffers clubbed together in one message */
+	vbuf->nent = (count + id_len + PAGE_SIZE - 1) / PAGE_SIZE;
+
+	vbuf->bufs = kzalloc(vbuf->nent, GFP_KERNEL);
+	if (!vbuf->bufs)
+		goto free_vbuf;
+
+	vbuf->sg = kzalloc(vbuf->nent, GFP_KERNEL);
+	if (!vbuf->sg)
+		goto free_bufs;
+	sg_init_table(vbuf->sg, vbuf->nent);
+
+	i = 0; /* vbuf->bufs[i] */
+	offset = 0; /* offset in the user buffer */
+	while (count - offset) {
+		size = min(count - offset + id_len, PAGE_SIZE);
+		vbuf->bufs[i] = kzalloc(size, GFP_KERNEL);
+		if (!vbuf->bufs[i]) {
+			ret = -EFBIG;
+			goto free_buffers;
+		}
+		if (id_len) {
+			memcpy(vbuf->bufs[i], &id, id_len);
+			size -= id_len;
+		}
+		ret = copy_from_user(vbuf->bufs[i] + id_len, ubuf + offset, size);
+		offset += size - ret;
+
+		sg_set_buf(&vbuf->sg[i], vbuf->bufs[i], size - ret + id_len);
+		id_len = 0; /* Pass the port id only in the first buffer */
+		i++;
+	}
+	if (out_vq->vq_ops->add_buf(out_vq, vbuf->sg, i, 0, vbuf)) {
+		/* XXX: We can't send the buffer. Report failure */
+		ret = 0;
+	}
+	/* Tell Host to go! */
+	out_vq->vq_ops->kick(out_vq);
+
+	/* We're expected to return the amount of data we wrote */
+	return offset;
+free_buffers:
+	while (i--)
+		kfree(vbuf->bufs[i]);
+	kfree(vbuf->sg);
+free_bufs:
+	kfree(vbuf->bufs);
+free_vbuf:
+	kfree(vbuf);
+	return ret;
+}
+
+static long virtserial_ioctl(struct file *filp, unsigned int ioctl,
+			     unsigned long arg)
+{
+	struct virtio_serial_port *port;
+	long ret;
+
+	port = filp->private_data;
+
+	ret = -EINVAL;
+	switch (ioctl) {
+	default:
+		break;
+	}
+	return ret;
+}
+
+static int virtserial_release(struct inode *inode, struct file *filp)
+{
+	filp->private_data = NULL;
+	return 0;
+}
+
+static int virtserial_open(struct inode *inode, struct file *filp)
+{
+	struct cdev *cdev = inode->i_cdev;
+	struct virtio_serial_port *port;
+
+	port = container_of(cdev, struct virtio_serial_port, cdev);
+
+	filp->private_data = port;
+	return 0;
+}
+
+static unsigned int virtserial_poll(struct file *filp, poll_table *wait)
+{
+	pr_notice("%s\n", __func__);
+	return 0;
+}
+
+static const struct file_operations virtserial_fops = {
+	.owner = THIS_MODULE,
+	.open  = virtserial_open,
+	.read  = virtserial_read,
+	.write = virtserial_write,
+	.compat_ioctl = virtserial_ioctl,
+	.unlocked_ioctl = virtserial_ioctl,
+	.poll  = virtserial_poll,
+	.release = virtserial_release,
+};
+
+static void virtio_serial_queue_work_handler(struct work_struct *work)
+{
+	struct scatterlist sg[1];
+	struct virtqueue *vq;
+	char *buf;
+
+	vq = virtserial.in_vq;
+	while (1) {
+		buf = kzalloc(PAGE_SIZE, GFP_KERNEL);
+		if (!buf)
+			break;
+
+		sg_init_one(sg, buf, PAGE_SIZE);
+
+		if (vq->vq_ops->add_buf(vq, sg, 0, 1, buf) < 0) {
+			kfree(buf);
+			break;
+		}
+	}
+	vq->vq_ops->kick(vq);
+}
+
+static void virtio_serial_rx_work_handler(struct work_struct *work)
+{
+	struct virtio_serial_port *port = NULL;
+	struct virtio_serial_port_buffer *buf;
+	struct virtqueue *vq;
+	char *tmpbuf;
+	unsigned int tmplen;
+
+	vq = virtserial.in_vq;
+	while ((tmpbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
+		port = get_port_from_buf(tmpbuf);
+		if (!port) {
+			/* No valid index at start of
+			 * buffer. Drop it.
+			 */
+			pr_debug("%s: invalid index in buffer, %c %d\n",
+				 __func__, tmpbuf[0], tmpbuf[0]);
+			break;
+		}
+		buf = kzalloc(sizeof(struct virtio_serial_port_buffer),
+			      GFP_KERNEL);
+		if (!buf)
+			break;
+
+		buf->buf = tmpbuf;
+		buf->len = tmplen;
+		buf->offset = sizeof(struct virtio_serial_id);
+		list_add_tail(&buf->list, &port->readbuf_head);
+
+		complete(&port->have_data);
+	}
+	/* Allocate buffers for all the ones that got used up */
+	schedule_work(&virtserial.queue_work);
+}
+
+static void virtio_serial_tx_work_handler(struct work_struct *work)
+{
+	struct virtqueue *vq;
+	struct vbuf *vbuf;
+	unsigned int tmplen;
+	int i;
+
+	vq = virtserial.out_vq;
+	while ((vbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
+		for (i = 0; i < vbuf->nent; i++) {
+			kfree(vbuf->bufs[i]);
+		}
+		kfree(vbuf->bufs);
+		kfree(vbuf->sg);
+		kfree(vbuf);
+	}
+}
+
+static void rx_intr(struct virtqueue *vq)
+{
+	schedule_work(&virtserial.rx_work);
+}
+
+static void tx_intr(struct virtqueue *vq)
+{
+	schedule_work(&virtserial.tx_work);
+}
+
+static void config_intr(struct virtio_device *vdev)
+{
+	schedule_work(&virtserial.config_work);
+}
+
+static u32 virtserial_get_hot_add_port(struct virtio_serial_config *config)
+{
+	u32 i;
+	u32 port_nr;
+
+	for (i = 0; i < virtserial.config->max_nr_ports / 32; i++) {
+		port_nr = ffs(config->ports_map[i] ^ virtserial.config->ports_map[i]);
+		if (port_nr)
+			break;
+	}
+	if (unlikely(!port_nr))
+		return VIRTIO_SERIAL_BAD_ID;
+
+	/* We used ffs above */
+	port_nr--;
+
+	/* FIXME: Do this only when add_port is successful */
+	virtserial.config->ports_map[i] |= 1U << port_nr;
+
+	port_nr += i * 32;
+	return port_nr;
+}
+
+static u32 virtserial_find_next_port(u32 *map, int *map_i)
+{
+	u32 port_nr;
+
+	while (1) {
+		port_nr = ffs(*map);
+		if (port_nr)
+			break;
+
+		if (unlikely(*map_i >= virtserial.config->max_nr_ports / 32))
+			return VIRTIO_SERIAL_BAD_ID;
+		++*map_i;
+		*map = virtserial.config->ports_map[*map_i];
+	}
+	/* We used ffs above */
+	port_nr--;
+
+	/* FIXME: Do this only when add_port is successful / reset bit
+	 * in config space if add_port was unsuccessful
+	 */
+	*map &= ~(1U << port_nr);
+
+	port_nr += *map_i * 32;
+	return port_nr;
+}
+
+static int virtserial_add_port(u32 port_nr)
+{
+	struct virtio_serial_port *port;
+	dev_t devt;
+	int ret;
+
+	port = kzalloc(sizeof(struct virtio_serial_port), GFP_KERNEL);
+	if (!port)
+		return -ENOMEM;
+
+	devt = MKDEV(major, port_nr);
+	cdev_init(&port->cdev, &virtserial_fops);
+
+	ret = register_chrdev_region(devt, 1, "virtio-serial");
+	if (ret < 0) {
+		pr_err("%s: error registering chrdev region, ret = %d\n",
+		       __func__, ret);
+		goto free_cdev;
+	}
+	ret = cdev_add(&port->cdev, devt, 1);
+	if (ret < 0) {
+		pr_err("%s: error adding cdev, ret = %d\n", __func__, ret);
+		goto free_cdev;
+	}
+	port->dev = device_create(virtserial.class, NULL, devt, NULL,
+				  "vmch%u", port_nr);
+	if (IS_ERR(port->dev)) {
+		ret = PTR_ERR(port->dev);
+		pr_err("%s: Error creating device, ret = %d\n", __func__, ret);
+		goto free_cdev;
+	}
+	INIT_LIST_HEAD(&port->readbuf_head);
+	init_completion(&port->have_data);
+
+	list_add_tail(&port->next, &virtserial.port_head);
+
+	pr_info("virtio-serial port found at id %u\n", port_nr);
+
+	return 0;
+free_cdev:
+	unregister_chrdev(major, "virtio-serial");
+	return ret;
+}
+
+static __u32 get_ports_map_size(__u32 max_ports)
+{
+	return sizeof(__u32) * ((max_ports + 31) / 32);
+}
+
+static void virtio_serial_config_work_handler(struct work_struct *work)
+{
+	struct virtio_serial_config *virtserconf;
+	struct virtio_device *vdev = virtserial.vdev;
+	u32 i, port_nr;
+	int ret;
+
+	virtserconf = kmalloc(sizeof(struct virtio_serial_config) +
+			      get_ports_map_size(virtserial.config->max_nr_ports),
+			      GFP_KERNEL);
+	vdev->config->get(vdev,
+			  offsetof(struct virtio_serial_config, nr_active_ports),
+			  &virtserconf->nr_active_ports,
+			  sizeof(virtserconf->nr_active_ports));
+	vdev->config->get(vdev,
+			  offsetof(struct virtio_serial_config, ports_map),
+			  virtserconf->ports_map,
+			  get_ports_map_size(virtserial.config->max_nr_ports));
+
+	/* Hot-add ports */
+	for (i = virtserial.config->nr_active_ports;
+	     i < virtserconf->nr_active_ports; i++) {
+		port_nr = virtserial_get_hot_add_port(virtserconf);
+		if (port_nr == VIRTIO_SERIAL_BAD_ID)
+			continue;
+		ret = virtserial_add_port(port_nr);
+		if (!ret)
+			virtserial.config->nr_active_ports++;
+	}
+	kfree(virtserconf);
+}
+
+static int virtserial_probe(struct virtio_device *vdev)
+{
+	struct virtqueue *vqs[3];
+	const char *vq_names[] = { "input", "output" };
+	vq_callback_t *vq_callbacks[] = { rx_intr, tx_intr };
+	u32 i, map;
+	int ret, map_i;
+	u32 max_nr_ports;
+
+	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
+					 max_nr_ports),
+			  &max_nr_ports,
+			  sizeof(max_nr_ports));
+	virtserial.config = kmalloc(sizeof(struct virtio_serial_config)
+				    + get_ports_map_size(max_nr_ports),
+				    GFP_KERNEL);
+	virtserial.config->max_nr_ports = max_nr_ports;
+
+	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
+					 nr_active_ports),
+			  &virtserial.config->nr_active_ports,
+			  sizeof(virtserial.config->nr_active_ports));
+	vdev->config->get(vdev,
+			  offsetof(struct virtio_serial_config, ports_map),
+			  virtserial.config->ports_map,
+			  get_ports_map_size(max_nr_ports));
+
+	virtserial.vdev = vdev;
+
+	ret = vdev->config->find_vqs(vdev, 2, vqs, vq_callbacks, vq_names);
+	if (ret)
+		goto fail;
+
+	virtserial.in_vq = vqs[0];
+	virtserial.out_vq = vqs[1];
+
+	INIT_LIST_HEAD(&virtserial.port_head);
+
+	map_i = 0;
+	map = virtserial.config->ports_map[map_i];
+	for (i = 0; i < virtserial.config->nr_active_ports; i++) {
+		__u32 port_nr;
+
+		port_nr = virtserial_find_next_port(&map, &map_i);
+		if (unlikely(port_nr == VIRTIO_SERIAL_BAD_ID))
+			continue;
+
+		virtserial_add_port(port_nr);
+	}
+	INIT_WORK(&virtserial.rx_work, &virtio_serial_rx_work_handler);
+	INIT_WORK(&virtserial.tx_work, &virtio_serial_tx_work_handler);
+	INIT_WORK(&virtserial.queue_work, &virtio_serial_queue_work_handler);
+	INIT_WORK(&virtserial.config_work, &virtio_serial_config_work_handler);
+
+	/* Allocate pages to fill the receive queue */
+	schedule_work(&virtserial.queue_work);
+
+	return 0;
+fail:
+	return ret;
+}
+
+
+static void virtserial_remove_port_data(struct virtio_serial_port *port)
+{
+	struct list_head *ptr, *ptr2;
+
+	device_destroy(virtserial.class, port->dev->devt);
+	unregister_chrdev_region(port->dev->devt, 1);
+	cdev_del(&port->cdev);
+
+	/* Remove the buffers in which we have unconsumed data */
+	list_for_each_safe(ptr, ptr2, &port->readbuf_head) {
+		struct virtio_serial_port_buffer *buf;
+
+		buf = list_entry(ptr, struct virtio_serial_port_buffer, list);
+
+		list_del(&buf->list);
+		kfree(buf->buf);
+		kfree(buf);
+	}
+}
+
+static void virtserial_remove(struct virtio_device *vdev)
+{
+	struct list_head *ptr, *ptr2;
+	char *buf;
+	int len;
+
+	unregister_chrdev(major, "virtio-serial");
+	class_destroy(virtserial.class);
+
+	cancel_work_sync(&virtserial.rx_work);
+
+	/* Free up the unused buffers in the receive queue */
+	while ((buf = virtserial.in_vq->vq_ops->get_buf(virtserial.in_vq, &len)))
+		kfree(buf);
+
+	vdev->config->del_vqs(vdev);
+
+	list_for_each_safe(ptr, ptr2, &virtserial.port_head) {
+		struct virtio_serial_port *port;
+
+		port = list_entry(ptr, struct virtio_serial_port, next);
+
+		list_del(&port->next);
+		virtserial_remove_port_data(port);
+		kfree(port);
+	}
+	kfree(virtserial.config);
+}
+
+static struct virtio_device_id id_table[] = {
+	{ VIRTIO_ID_SERIAL, VIRTIO_DEV_ANY_ID },
+	{ 0 },
+};
+
+static struct virtio_driver virtio_serial = {
+  //	.feature_table = features,
+  //	.feature_table_size = ARRAY_SIZE(features),
+	.driver.name =	KBUILD_MODNAME,
+	.driver.owner =	THIS_MODULE,
+	.id_table =	id_table,
+	.probe =	virtserial_probe,
+	.remove =	virtserial_remove,
+	.config_changed = config_intr,
+};
+
+static int __init init(void)
+{
+	int ret;
+
+	virtserial.class = class_create(THIS_MODULE, "virtio-serial");
+	if (IS_ERR(virtserial.class)) {
+		pr_err("Error creating virtio-serial class\n");
+		ret = PTR_ERR(virtserial.class);
+		return ret;
+	}
+	ret = register_virtio_driver(&virtio_serial);
+	if (ret) {
+		class_destroy(virtserial.class);
+		return ret;
+	}
+	return 0;
+}
+
+static void __exit fini(void)
+{
+	unregister_virtio_driver(&virtio_serial);
+}
+module_init(init);
+module_exit(fini);
+
+MODULE_DEVICE_TABLE(virtio, id_table);
+MODULE_DESCRIPTION("Virtio serial driver");
+MODULE_LICENSE("GPL");
diff --git a/include/linux/virtio_serial.h b/include/linux/virtio_serial.h
new file mode 100644
index 0000000..025dcf1
--- /dev/null
+++ b/include/linux/virtio_serial.h
@@ -0,0 +1,27 @@
+#ifndef _LINUX_VIRTIO_SERIAL_H
+#define _LINUX_VIRTIO_SERIAL_H
+#include <linux/types.h>
+#include <linux/virtio_config.h>
+
+/* Guest kernel - Host interface */
+
+/* The ID for virtio serial */
+#define VIRTIO_ID_SERIAL		7
+
+#define VIRTIO_SERIAL_BAD_ID		(~(u32)0)
+
+struct virtio_serial_config {
+	__u32 max_nr_ports;
+	__u32 nr_active_ports;
+	__u32 ports_map[0 /* (max_nr_ports + 31) / 32 */];
+};
+
+#ifdef __KERNEL__
+
+/* Guest kernel - Guest userspace interface */
+
+/* IOCTL-related */
+#define VIRTIO_SERIAL_IO 0xAF
+
+#endif /* __KERNEL__ */
+#endif /* _LINUX_VIRTIO_SERIAL_H */
-- 
1.6.2.5

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

* [Qemu-devel] [PATCH 1/3] virtio-serial: virtio device for simple host <-> guest communication
  2009-07-27 18:04 ` [Qemu-devel] [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Amit Shah
@ 2009-07-27 18:04   ` Amit Shah
  2009-07-27 18:04     ` [Qemu-devel] [PATCH 2/3] vnc: add a is_vnc_active() helper Amit Shah
  2009-08-05  0:03   ` [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Rusty Russell
  1 sibling, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-07-27 18:04 UTC (permalink / raw)
  To: virtualization; +Cc: Amit Shah, qemu-devel, kvm

This interface presents a char device from which bits can be
sent and read.

Sample uses for such a device can be obtaining info from the
guest like the file systems used, apps installed, etc. for
offline usage and logged-in users, clipboard copy-paste, etc.
for online usage.

Each port is to be assigned a unique function, for example, the first
4 ports may be reserved for libvirt usage, the next 4 for generic
streaming data and so on. This port-function mapping isn't finalised
yet.

For requirements, use-cases and some history see

    http://www.linux-kvm.org/page/VMchannel_Requirements

Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
 Makefile.target    |    2 +-
 hw/pc.c            |   17 +++
 hw/pci.h           |    1 +
 hw/qdev.c          |    6 +-
 hw/virtio-pci.c    |   17 +++
 hw/virtio-serial.c |  367 ++++++++++++++++++++++++++++++++++++++++++++++++++++
 hw/virtio-serial.h |   36 +++++
 hw/virtio.h        |    1 +
 monitor.c          |    7 +
 qemu-monitor.hx    |   10 ++
 qemu-options.hx    |    8 +
 sysemu.h           |   13 ++
 vl.c               |   90 +++++++++++++
 13 files changed, 573 insertions(+), 2 deletions(-)
 create mode 100644 hw/virtio-serial.c
 create mode 100644 hw/virtio-serial.h

diff --git a/Makefile.target b/Makefile.target
index df1f32b..21e8ec3 100644
--- a/Makefile.target
+++ b/Makefile.target
@@ -538,7 +538,7 @@ obj-y = vl.o osdep.o monitor.o pci.o loader.o isa_mmio.o machine.o \
         gdbstub.o gdbstub-xml.o msix.o ioport.o
 # virtio has to be here due to weird dependency between PCI and virtio-net.
 # need to fix this properly
-obj-y += virtio-blk.o virtio-balloon.o virtio-net.o virtio-console.o
+obj-y += virtio-blk.o virtio-balloon.o virtio-net.o virtio-console.o virtio-serial.o
 obj-$(CONFIG_KVM) += kvm.o kvm-all.o
 
 LIBS+=-lz
diff --git a/hw/pc.c b/hw/pc.c
index ecc7046..6badad9 100644
--- a/hw/pc.c
+++ b/hw/pc.c
@@ -40,6 +40,8 @@
 
 #include "qemu-kvm.h"
 
+void *virtio_serial_new_port(PCIDevice *dev, int idx);
+
 /* output Bochs bios info messages */
 //#define DEBUG_BIOS
 
@@ -1456,6 +1458,21 @@ static void pc_init1(ram_addr_t ram_size,
         }
     }
 
+    /* Add virtio serial devices */
+    if (pci_enabled && virtio_serial_nr_ports) {
+        void *dev;
+
+        dev = pci_create_simple(pci_bus, -1, "virtio-serial-pci");
+        if (!dev) {
+            fprintf(stderr, "qemu: could not create virtio serial pci device\n");
+            exit(1);
+        }
+
+        for (i = 0; i < virtio_serial_nr_ports; i++) {
+                virtio_serial_new_port(dev, virtio_serial_idx[i]);
+        }
+    }
+
 #ifdef USE_KVM_DEVICE_ASSIGNMENT
     if (kvm_enabled()) {
         add_assigned_devices(pci_bus, assigned_devices, assigned_devices_index);
diff --git a/hw/pci.h b/hw/pci.h
index 7ca3ba9..42d1291 100644
--- a/hw/pci.h
+++ b/hw/pci.h
@@ -74,6 +74,7 @@ extern target_phys_addr_t pci_mem_base;
 #define PCI_DEVICE_ID_VIRTIO_BLOCK       0x1001
 #define PCI_DEVICE_ID_VIRTIO_BALLOON     0x1002
 #define PCI_DEVICE_ID_VIRTIO_CONSOLE     0x1003
+#define PCI_DEVICE_ID_VIRTIO_SERIAL      0x1004
 
 typedef void PCIConfigWriteFunc(PCIDevice *pci_dev,
                                 uint32_t address, uint32_t data, int len);
diff --git a/hw/qdev.c b/hw/qdev.c
index 83e98bf..f2ab508 100644
--- a/hw/qdev.c
+++ b/hw/qdev.c
@@ -156,9 +156,13 @@ CharDriverState *qdev_init_chardev(DeviceState *dev)
 {
     static int next_serial;
     static int next_virtconsole;
+    static int next_virtserial;
+
     /* FIXME: This is a nasty hack that needs to go away.  */
-    if (strncmp(dev->info->name, "virtio", 6) == 0) {
+    if (strncmp(dev->info->name, "virtio-console", 14) == 0) {
         return virtcon_hds[next_virtconsole++];
+    } else if (strncmp(dev->info->name, "virtio-serial", 13) == 0) {
+        return virtio_serial_hds[next_virtserial++];
     } else {
         return serial_hds[next_serial++];
     }
diff --git a/hw/virtio-pci.c b/hw/virtio-pci.c
index 3b9bfd1..53e4295 100644
--- a/hw/virtio-pci.c
+++ b/hw/virtio-pci.c
@@ -478,6 +478,19 @@ static void virtio_balloon_init_pci(PCIDevice *pci_dev)
                     0x00);
 }
 
+static void virtio_serial_init_pci(PCIDevice *pci_dev)
+{
+    VirtIOPCIProxy *proxy = DO_UPCAST(VirtIOPCIProxy, pci_dev, pci_dev);
+    VirtIODevice *vdev;
+
+    vdev = virtio_serial_init(&pci_dev->qdev);
+    virtio_init_pci(proxy, vdev,
+                    PCI_VENDOR_ID_REDHAT_QUMRANET,
+                    PCI_DEVICE_ID_VIRTIO_SERIAL,
+                    PCI_CLASS_COMMUNICATION_OTHER,
+                    0x00);
+}
+
 static PCIDeviceInfo virtio_info[] = {
     {
         .qdev.name = "virtio-blk-pci",
@@ -496,6 +509,10 @@ static PCIDeviceInfo virtio_info[] = {
         .qdev.size = sizeof(VirtIOPCIProxy),
         .init      = virtio_balloon_init_pci,
     },{
+        .qdev.name = "virtio-serial-pci",
+        .qdev.size = sizeof(VirtIOPCIProxy),
+        .init      = virtio_serial_init_pci,
+    },{
         /* end of list */
     }
 };
diff --git a/hw/virtio-serial.c b/hw/virtio-serial.c
new file mode 100644
index 0000000..d77af9b
--- /dev/null
+++ b/hw/virtio-serial.c
@@ -0,0 +1,367 @@
+/*
+ * Virtio serial interface
+ *
+ * This serial interface is a paravirtualised guest<->host
+ * communication channel for relaying short messages and events in
+ * either direction.
+ *
+ * There's support for multiple serial channels within one virtio PCI
+ * device to keep the guest PCI device count low.
+ *
+ * Copyright (C) 2009, Red Hat, Inc.
+ *
+ * Author(s): Amit Shah <amit.shah@redhat.com>
+ *
+ * This work is licensed under the terms of the GNU GPL, version 2.  See
+ * the COPYING file in the top-level directory.
+ */
+
+#include "hw.h"
+#include "pci.h"
+#include "monitor.h"
+#include "qemu-char.h"
+#include "virtio.h"
+#include "virtio-serial.h"
+
+typedef struct VirtIOSerial {
+    PCIDevice *dev;
+    VirtIODevice *vdev;
+    VirtQueue *ivq, *ovq;
+    struct VirtIOSerialPort *ports;
+    struct virtio_serial_config config;
+} VirtIOSerial;
+
+typedef struct VirtIOSerialPort {
+    VirtIOSerial *virtserial;
+    CharDriverState *hd;
+} VirtIOSerialPort;
+
+typedef struct VirtIOSerialId {
+    uint32_t id; /* Port id */
+} VirtIOSerialId;
+
+VirtIOSerial virtio_serial;
+
+static VirtIOSerialPort *get_port_from_id(uint32_t id)
+{
+    if (id > MAX_VIRTIO_SERIAL_PORTS)
+        return NULL;
+
+    return &virtio_serial.ports[id];
+}
+
+static int get_id_from_port(VirtIOSerialPort *port)
+{
+    uint32_t i;
+
+    for (i = 0; i < MAX_VIRTIO_SERIAL_PORTS; i++) {
+        if (port == &virtio_serial.ports[i]) {
+            return i;
+        }
+    }
+    return VIRTIO_SERIAL_BAD_ID;
+}
+
+void virtio_serial_monitor_command(Monitor *mon,
+                            const char *command, const char *param)
+{
+    VirtIOSerialPort *port = &virtio_serial.ports[0];
+    VirtIODevice *vdev = port->virtserial->vdev;
+    VirtQueue *vq = port->virtserial->ivq;
+    VirtQueueElement elem;
+    char buf[300];
+    ssize_t len;
+    unsigned int i;
+    int ret;
+
+    if(!strncmp(command, "add_port", 8)) {
+        if (!param) {
+            monitor_printf(mon, "Error: need port id to add new port\n");
+            return;
+        }
+        ret = init_virtio_serial_port(virtio_serial_nr_ports, param);
+        if (ret < 0) {
+            monitor_printf(mon, "Error: cannot add new port: %s\n",
+                           strerror(-ret));
+            return;
+        }
+        virtio_serial_new_port(NULL, virtio_serial_idx[virtio_serial_nr_ports]);
+        virtio_serial_nr_ports++;
+        virtio_serial.config.nr_active_ports = cpu_to_le32(virtio_serial_nr_ports);
+        return;
+    }
+    if (!virtio_queue_ready(vq)) {
+        goto queue_not_ready;
+    }
+    len = snprintf(buf, 299, "%s %s\n", command, param);
+
+    ret = virtqueue_pop(vq, &elem);
+    if (!ret) {
+        goto queue_not_ready;
+    }
+    i = 0;
+    /* Note: We only have PAGE_SIZE sized buffers */
+    memcpy(elem.in_sg[i].iov_base, buf, len);
+    elem.in_sg[i].iov_len = len;
+
+    virtqueue_push(vq, &elem, len);
+    virtio_notify(vdev, vq);
+    return;
+
+queue_not_ready:
+    monitor_printf(cur_mon, "No free virtio buffer found. Message not sent.\n");
+    return;
+}
+
+static size_t flush_buf(uint32_t id, const uint8_t *buf, size_t len)
+{
+    VirtIOSerialPort *port;
+    size_t write_len = 0;
+
+    port = get_port_from_id(id);
+    if (port && port->hd) {
+        write_len = qemu_chr_write(port->hd, buf, len);
+	return write_len;
+    }
+    return 0;
+}
+
+/* Guest wrote something to some port.
+ *
+ * Flush the data in the entire chunk that we received rather than
+ * splitting it into multiple buffers. VNC clients don't consume split
+ * buffers
+ */
+static void virtio_serial_handle_output(VirtIODevice *vdev, VirtQueue *vq)
+{
+    VirtQueueElement elem;
+
+    while (virtqueue_pop(vq, &elem)) {
+        uint8_t *buf;
+        size_t totlen, buf_offset;
+        VirtIOSerialId id;
+        int id_len, i;
+
+        id_len = sizeof(id);
+        memcpy(&id, elem.out_sg[0].iov_base, id_len);
+
+        totlen = 0;
+        for (i = 0; i < elem.out_num; i++) {
+            totlen += elem.out_sg[i].iov_len;
+        }
+        buf = qemu_mallocz(totlen - id_len);
+        buf_offset = 0;
+        for (i = 0; i < elem.out_num; i++) {
+            size_t len = elem.out_sg[i].iov_len - id_len;
+
+            memcpy(buf + buf_offset, elem.out_sg[i].iov_base + id_len, len);
+
+            buf_offset += len;
+            id_len = 0; /* id is only sent with the first packet in each sg */
+        }
+        flush_buf(id.id, buf, totlen - sizeof(id_len));
+        virtqueue_push(vq, &elem, totlen);
+        qemu_free(buf);
+    }
+    virtio_notify(vdev, vq);
+}
+
+static void virtio_serial_handle_input(VirtIODevice *vdev, VirtQueue *vq)
+{
+}
+
+static void write_to_port(VirtIOSerialPort *port,
+			  const uint8_t *buf, size_t size)
+{
+    VirtQueue *vq = port->virtserial->ivq;
+    VirtQueueElement elem;
+    size_t offset = 0;
+    size_t len = 0;
+
+    if (!virtio_queue_ready(vq)) {
+        return;
+    }
+    while (offset < size) {
+        VirtIOSerialId id;
+        int i, id_len;
+
+        id_len = sizeof(VirtIOSerialId);
+
+        if (!virtqueue_pop(vq, &elem)) {
+            break;
+        }
+        if (elem.in_sg[0].iov_len < id_len) {
+            /* We can't even store our port number in this buffer. Bug? */
+            fprintf(stderr, "virtio-serial: size %zd less than expected\n",
+                    elem.in_sg[0].iov_len);
+            exit(1);
+        }
+        id.id = cpu_to_le32(get_id_from_port(port));
+        memcpy(elem.in_sg[0].iov_base, &id, id_len);
+
+        for (i = 0; offset < size && i < elem.in_num; i++) {
+            len = MIN(elem.in_sg[i].iov_len - id_len, size - offset);
+
+            memcpy(elem.in_sg[i].iov_base + id_len, buf + offset, len);
+            offset += len;
+            id_len = 0;
+        }
+        virtqueue_push(vq, &elem, len + sizeof(VirtIOSerialId));
+    }
+    virtio_notify(port->virtserial->vdev, vq);
+}
+
+/* Readiness of the guest to accept data on a port */
+static int cons_can_read(void *opaque)
+{
+    VirtIOSerialPort *port = (VirtIOSerialPort *) opaque;
+    VirtQueue *vq = port->virtserial->ivq;
+    int size;
+
+    if (!virtio_queue_ready(vq)) {
+        return 0;
+    }
+    size = TARGET_PAGE_SIZE;
+    if (virtqueue_avail_bytes(vq, size, 0)) {
+        return size - sizeof(VirtIOSerialId);
+    }
+    size = sizeof(VirtIOSerialId) + 1;
+    if (virtqueue_avail_bytes(vq, size, 0)) {
+        return size - sizeof(VirtIOSerialId);
+    }
+    return 0;
+}
+
+/* Send data from a char device over to the guest */
+static void cons_read(void *opaque, const uint8_t *buf, int size)
+{
+    VirtIOSerialPort *port = (VirtIOSerialPort *) opaque;
+
+    write_to_port(port, buf, size);
+}
+
+static void cons_event(void *opaque, int event)
+{
+    /* we will ignore any event for the time being */
+}
+
+static uint32_t virtio_serial_get_features(VirtIODevice *vdev)
+{
+    return 0;
+}
+
+/* Guest requested config info */
+static void virtio_serial_get_config(VirtIODevice *vdev, uint8_t *config_data)
+{
+    memcpy(config_data, &virtio_serial.config, sizeof(virtio_serial.config));
+}
+
+static void virtio_serial_set_config(VirtIODevice *vdev,
+                                     const uint8_t *config_data)
+{
+    struct virtio_serial_config config;
+
+    memcpy(&config, config_data, sizeof(config));
+}
+
+static void virtio_serial_save(QEMUFile *f, void *opaque)
+{
+    VirtIODevice *vdev = opaque;
+
+    virtio_save(vdev, f);
+}
+
+static int virtio_serial_load(QEMUFile *f, void *opaque, int version_id)
+{
+    VirtIODevice *vdev = opaque;
+
+    if (version_id != 1)
+        return -EINVAL;
+
+    virtio_load(vdev, f);
+    return 0;
+}
+
+void virtio_serial_set_port_active(uint32_t idx)
+{
+    int i = 0;
+
+    while (idx / 32) {
+        i++;
+        idx -= 32;
+    }
+    virtio_serial.config.ports_map[i] |= 1U << idx;
+}
+
+bool virtio_serial_is_port_active(uint32_t idx)
+{
+    int i = 0;
+
+    while (idx / 32) {
+        i++;
+        idx -= 32;
+    }
+    return virtio_serial.config.ports_map[i] & (1U << idx);
+}
+
+void *virtio_serial_new_port(PCIDevice *dev, int idx)
+{
+    VirtIOSerialPort *port;
+
+    port = get_port_from_id(idx);
+    port->virtserial = &virtio_serial;
+
+    if (!port->virtserial->dev && !dev) {
+        return NULL;
+    }
+    if (!port->virtserial->dev) {
+        port->virtserial->dev = dev;
+    }
+    /* Hot-adding ports to existing device */
+    if (!dev) {
+        dev = port->virtserial->dev;
+    }
+    port->hd = qdev_init_chardev(&dev->qdev);
+    if (port->hd) {
+        qemu_chr_add_handlers(port->hd, cons_can_read, cons_read, cons_event,
+                              port);
+    }
+    /* Send an update to the guest about this new port added */
+    virtio_notify_config(port->virtserial->vdev);
+    return port;
+}
+
+VirtIODevice *virtio_serial_init(DeviceState *dev)
+{
+    VirtIODevice *vdev;
+
+    vdev = virtio_common_init("virtio-serial",
+			      VIRTIO_ID_SERIAL,
+			      sizeof(struct virtio_serial_config),
+			      sizeof(VirtIODevice));
+    if (vdev == NULL)
+        return NULL;
+
+    virtio_serial.vdev = vdev;
+    vdev->get_config = virtio_serial_get_config;
+    vdev->set_config = virtio_serial_set_config;
+    vdev->get_features = virtio_serial_get_features;
+
+    /* Add queue for host to guest transfers */
+    virtio_serial.ivq = virtio_add_queue(vdev, VIRTQUEUE_MAX_SIZE,
+                                         virtio_serial_handle_input);
+    /* Add queue for guest to host transfers */
+    virtio_serial.ovq = virtio_add_queue(vdev, VIRTQUEUE_MAX_SIZE,
+                                         virtio_serial_handle_output);
+
+    /* Allocate space for the max. number of serial ports supported */
+    virtio_serial.ports = qemu_mallocz(sizeof(VirtIOSerialPort) * MAX_VIRTIO_SERIAL_PORTS);
+
+    register_savevm("virtio-serial", -1, 1, virtio_serial_save,
+                    virtio_serial_load, vdev);
+
+    virtio_serial.config.max_nr_ports = cpu_to_le32(MAX_VIRTIO_SERIAL_PORTS);
+    virtio_serial.config.nr_active_ports = cpu_to_le32(virtio_serial_nr_ports);
+
+    return vdev;
+}
diff --git a/hw/virtio-serial.h b/hw/virtio-serial.h
new file mode 100644
index 0000000..08c4b51
--- /dev/null
+++ b/hw/virtio-serial.h
@@ -0,0 +1,36 @@
+/*
+ * Virtio Serial Support
+ *
+ * Copyright (C) 2009, Red Hat, Inc.
+ *
+ * Author(s): Amit Shah <amit.shah@redhat.com>
+ *
+ * This work is licensed under the terms of the GNU GPL, version 2.  See
+ * the COPYING file in the top-level directory.
+ *
+ */
+#ifndef _QEMU_VIRTIO_SERIAL_H
+#define _QEMU_VIRTIO_SERIAL_H
+
+#include "sysemu.h"
+
+/* The ID for virtio serial */
+#define VIRTIO_ID_SERIAL		7
+#define VIRTIO_SERIAL_BAD_ID		(~(uint32_t)0)
+
+struct virtio_serial_config
+{
+    uint32_t max_nr_ports;
+    uint32_t nr_active_ports;
+    uint32_t ports_map[(MAX_VIRTIO_SERIAL_PORTS + 31) / 32];
+};
+
+/* Some defines for the control channel key */
+#define VIRTIO_SERIAL_GET_PORT_NAME 1
+
+
+void *virtio_serial_new_port(PCIDevice *dev, int idx);
+void virtio_serial_monitor_command(Monitor *mon,
+                                   const char *command, const char *param);
+
+#endif
diff --git a/hw/virtio.h b/hw/virtio.h
index aa55677..ec111f6 100644
--- a/hw/virtio.h
+++ b/hw/virtio.h
@@ -165,5 +165,6 @@ VirtIODevice *virtio_blk_init(DeviceState *dev);
 VirtIODevice *virtio_net_init(DeviceState *dev);
 VirtIODevice *virtio_console_init(DeviceState *dev);
 VirtIODevice *virtio_balloon_init(DeviceState *dev);
+VirtIODevice *virtio_serial_init(DeviceState *dev);
 
 #endif
diff --git a/monitor.c b/monitor.c
index 193f0b9..1e701e5 100644
--- a/monitor.c
+++ b/monitor.c
@@ -45,6 +45,7 @@
 #include "kvm.h"
 #include "acl.h"
 #include "exec-all.h"
+#include "hw/virtio-serial.h"
 
 #include "qemu-kvm.h"
 
@@ -1723,6 +1724,12 @@ static void do_inject_mce(Monitor *mon,
 }
 #endif
 
+static void do_virtio_serial_action(Monitor *mon,
+                                    const char *command, const char *param)
+{
+    virtio_serial_monitor_command(mon, command, param);
+}
+
 static const mon_cmd_t mon_cmds[] = {
 #include "qemu-monitor.h"
     { NULL, NULL, },
diff --git a/qemu-monitor.hx b/qemu-monitor.hx
index 1b0a6ef..2c30335 100644
--- a/qemu-monitor.hx
+++ b/qemu-monitor.hx
@@ -569,6 +569,16 @@ STEXI
 Change watchdog action.
 ETEXI
 
+    { "virtio-serial", "ss?", do_virtio_serial_action,
+      "<command> [<parameters>]\n",
+      "virtio-serial write port=3,key=get,value=clipboard\n"
+      "virtio-serial add_port\n"
+      "virtio-serial add_port port=6,name=foo,protocol=keyvalue\n" },
+STEXI
+@item virtio-serial
+Hot-add ports or send data to virtio-serial port
+ETEXI
+
     { "acl_show", "s", do_acl_show, "aclname",
       "list rules in the access control list" },
 STEXI
diff --git a/qemu-options.hx b/qemu-options.hx
index 8e6cd43..86c7441 100644
--- a/qemu-options.hx
+++ b/qemu-options.hx
@@ -1591,6 +1591,14 @@ STEXI
 Set virtio console.
 ETEXI
 
+DEF("virtioserial", HAS_ARG, QEMU_OPTION_virtioserial, \
+    "-virtioserial c\n" \
+    "                define virtio serial device\n")
+STEXI
+@item -virtserial @var{c}
+Set virtio serial device.
+ETEXI
+
 DEF("show-cursor", 0, QEMU_OPTION_show_cursor, \
     "-show-cursor    show cursor\n")
 STEXI
diff --git a/sysemu.h b/sysemu.h
index 75ea191..8dc5847 100644
--- a/sysemu.h
+++ b/sysemu.h
@@ -2,6 +2,7 @@
 #define SYSEMU_H
 /* Misc. things related to the system emulator.  */
 
+#include <stdbool.h>
 #include "qemu-common.h"
 
 #ifdef _WIN32
@@ -240,6 +241,18 @@ extern CharDriverState *parallel_hds[MAX_PARALLEL_PORTS];
 
 extern CharDriverState *virtcon_hds[MAX_VIRTIO_CONSOLES];
 
+/* virtio serial ports */
+
+#define MAX_VIRTIO_SERIAL_PORTS 64
+#define VIRTIO_SERIAL_PROTO_MAX_LEN 30
+
+extern CharDriverState *virtio_serial_hds[MAX_VIRTIO_SERIAL_PORTS];
+extern uint32_t virtio_serial_idx[MAX_VIRTIO_SERIAL_PORTS];
+extern int virtio_serial_nr_ports;
+extern int init_virtio_serial_port(int port, const char *opts);
+extern void virtio_serial_set_port_active(uint32_t idx);
+extern bool virtio_serial_is_port_active(uint32_t idx);
+
 #define TFR(expr) do { if ((expr) != -1) break; } while (errno == EINTR)
 
 #ifdef NEED_CPU_H
diff --git a/vl.c b/vl.c
index 30c4ff9..6c8aa11 100644
--- a/vl.c
+++ b/vl.c
@@ -221,6 +221,10 @@ int no_quit = 0;
 CharDriverState *serial_hds[MAX_SERIAL_PORTS];
 CharDriverState *parallel_hds[MAX_PARALLEL_PORTS];
 CharDriverState *virtcon_hds[MAX_VIRTIO_CONSOLES];
+CharDriverState *virtio_serial_hds[MAX_VIRTIO_SERIAL_PORTS];
+char virtio_serial_proto[MAX_VIRTIO_SERIAL_PORTS][VIRTIO_SERIAL_PROTO_MAX_LEN];
+uint32_t virtio_serial_idx[MAX_VIRTIO_SERIAL_PORTS];
+int virtio_serial_nr_ports;
 #ifdef TARGET_I386
 int win2k_install_hack = 0;
 int rtc_td_hack = 0;
@@ -4824,6 +4828,60 @@ char *qemu_find_file(int type, const char *name)
     return buf;
 }
 
+
+int init_virtio_serial_port(int port, const char *opts)
+{
+    char serdev[256];
+    const char *virtserprot;
+    const char *virtseridx;
+    uint32_t port_nr;
+    int j, k;
+
+    memset(serdev, 0, sizeof(serdev));
+    virtserprot = strstr(opts, ",protocol=");
+    virtseridx  = strstr(opts, ",id=");
+
+    /* We only create ports at specific locations */
+    if (!virtseridx) {
+        return -EINVAL;
+    }
+
+    j = k = 0;
+
+    /* Just to maintain compatibility with other qemu options,
+     * we have the format of
+     *
+     * -virtioserial unix:/tmp/foo,protocol=bar,id=3
+     *
+     * so to parse the 'unix:', we have to do the following
+     */
+    while (&opts[j] != virtserprot && &opts[j] != virtseridx) {
+        serdev[k++] = opts[j++];
+    }
+
+    port_nr = atol(virtseridx + 4); /* skip ',id=' */
+
+    k = virtio_serial_is_port_active(port_nr);
+    if (k || port_nr >= MAX_VIRTIO_SERIAL_PORTS) {
+        return -EEXIST;
+    }
+    if (serdev[0] && strncmp(serdev, "none", 4)) {
+        char label[32];
+        snprintf(label, sizeof(label), "virtio-serial%u", port_nr);
+        virtio_serial_hds[port] = qemu_chr_open(label, serdev, NULL);
+        if (!virtio_serial_hds[port]) {
+            /* We could be called from a monitor command to hot-add
+             * the port. Don't exit.
+             */
+            return -EIO;
+        }
+    }
+    virtio_serial_idx[port] = port_nr;
+    virtio_serial_set_port_active(port_nr);
+
+    return 0;
+}
+
 int main(int argc, char **argv, char **envp)
 {
     const char *gdbstub_dev = NULL;
@@ -4851,6 +4909,7 @@ int main(int argc, char **argv, char **envp)
     int parallel_device_index;
     const char *virtio_consoles[MAX_VIRTIO_CONSOLES];
     int virtio_console_index;
+    const char *virtio_serials[MAX_VIRTIO_SERIAL_PORTS];
     const char *loadvm = NULL;
     QEMUMachine *machine;
     const char *cpu_model;
@@ -4930,6 +4989,10 @@ int main(int argc, char **argv, char **envp)
         virtio_consoles[i] = NULL;
     virtio_console_index = 0;
 
+    for (i = 0; i < MAX_VIRTIO_SERIAL_PORTS; i++)
+        virtio_serials[i] = NULL;
+    virtio_serial_nr_ports = 0;
+
     for (i = 0; i < MAX_NODES; i++) {
         node_mem[i] = 0;
         node_cpumask[i] = 0;
@@ -5360,6 +5423,14 @@ int main(int argc, char **argv, char **envp)
                 virtio_consoles[virtio_console_index] = optarg;
                 virtio_console_index++;
                 break;
+	    case QEMU_OPTION_virtioserial:
+                if (virtio_serial_nr_ports >= MAX_VIRTIO_SERIAL_PORTS) {
+                    fprintf(stderr, "qemu: too many virtio serial ports\n");
+                    exit(1);
+                }
+                virtio_serials[virtio_serial_nr_ports] = optarg;
+                virtio_serial_nr_ports++;
+                break;
             case QEMU_OPTION_parallel:
                 if (parallel_device_index >= MAX_PARALLEL_PORTS) {
                     fprintf(stderr, "qemu: too many parallel ports\n");
@@ -5985,6 +6056,16 @@ int main(int argc, char **argv, char **envp)
         }
     }
 
+    for (i = 0; i < virtio_serial_nr_ports; i++) {
+        int ret;
+
+        ret = init_virtio_serial_port(i, virtio_serials[i]);
+        if (ret < 0) {
+            fprintf(stderr, "qemu: could not init virtio serial ports\n");
+            exit(1);
+        }
+    }
+
     module_call_init(MODULE_INIT_DEVICE);
 
     machine->init(ram_size, boot_devices,
@@ -6102,6 +6183,15 @@ int main(int argc, char **argv, char **envp)
         }
     }
 
+    for(i = 0; i < MAX_VIRTIO_SERIAL_PORTS; i++) {
+        const char *devname = virtio_serials[i];
+        if (virtio_serial_hds[i] && devname) {
+            if (strstart(devname, "vc", 0)) {
+                qemu_chr_printf(virtio_serial_hds[i], "virtio serial%d\r\n", i);
+            }
+        }
+    }
+
     if (gdbstub_dev && gdbserver_start(gdbstub_dev) < 0) {
         fprintf(stderr, "qemu: could not open gdbserver on device '%s'\n",
                 gdbstub_dev);
-- 
1.6.2.5

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

* [Qemu-devel] [PATCH 2/3] vnc: add a is_vnc_active() helper
  2009-07-27 18:04   ` [Qemu-devel] [PATCH 1/3] virtio-serial: virtio device for simple host <-> guest communication Amit Shah
@ 2009-07-27 18:04     ` Amit Shah
  2009-07-27 18:04       ` [Qemu-devel] [PATCH 3/3] virtio-serial: vnc: support for sending / receiving guest clipboard Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-07-27 18:04 UTC (permalink / raw)
  To: virtualization; +Cc: Amit Shah, qemu-devel, kvm

This helper is introduced to query the status of vnc.

Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
 vnc.c |   10 +++++++++-
 vnc.h |    2 +-
 2 files changed, 10 insertions(+), 2 deletions(-)

diff --git a/vnc.c b/vnc.c
index de0ff87..e4e78dc 100644
--- a/vnc.c
+++ b/vnc.c
@@ -176,9 +176,17 @@ static void do_info_vnc_client(Monitor *mon, VncState *client)
 #endif
 }
 
-void do_info_vnc(Monitor *mon)
+int is_vnc_active(void)
 {
     if (vnc_display == NULL || vnc_display->display == NULL) {
+        return 0;
+    }
+    return 1;
+}
+
+void do_info_vnc(Monitor *mon)
+{
+    if (!is_vnc_active()) {
         monitor_printf(mon, "Server: disabled\n");
     } else {
         char *serverAddr = vnc_socket_local_addr("     address: %s:%s\n",
diff --git a/vnc.h b/vnc.h
index 3ae95f3..9739c35 100644
--- a/vnc.h
+++ b/vnc.h
@@ -313,7 +313,7 @@ void buffer_append(Buffer *buffer, const void *data, size_t len);
 
 /* Misc helpers */
 
+int is_vnc_active(void);
 char *vnc_socket_local_addr(const char *format, int fd);
 char *vnc_socket_remote_addr(const char *format, int fd);
-
 #endif /* __QEMU_VNC_H */
-- 
1.6.2.5

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

* [Qemu-devel] [PATCH 3/3] virtio-serial: vnc: support for sending / receiving guest clipboard
  2009-07-27 18:04     ` [Qemu-devel] [PATCH 2/3] vnc: add a is_vnc_active() helper Amit Shah
@ 2009-07-27 18:04       ` Amit Shah
  0 siblings, 0 replies; 62+ messages in thread
From: Amit Shah @ 2009-07-27 18:04 UTC (permalink / raw)
  To: virtualization; +Cc: Amit Shah, qemu-devel, kvm

- Send ServerCutText message over to the vnc client from qemu
  on every write to the clipboard port via virtio-serial

- On receiving ClientCutText message send over the data to the
  guest via virtio-serial.

Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
 hw/virtio-serial.c |   15 +++++++++++++++
 hw/virtio-serial.h |    4 ++++
 vnc.c              |   24 ++++++++++++++++++++++++
 vnc.h              |    2 ++
 4 files changed, 45 insertions(+), 0 deletions(-)

diff --git a/hw/virtio-serial.c b/hw/virtio-serial.c
index d77af9b..95c715f 100644
--- a/hw/virtio-serial.c
+++ b/hw/virtio-serial.c
@@ -20,6 +20,7 @@
 #include "pci.h"
 #include "monitor.h"
 #include "qemu-char.h"
+#include "vnc.h"
 #include "virtio.h"
 #include "virtio-serial.h"
 
@@ -118,6 +119,10 @@ static size_t flush_buf(uint32_t id, const uint8_t *buf, size_t len)
     VirtIOSerialPort *port;
     size_t write_len = 0;
 
+    if (id == VIRTIO_SERIAL_CLIPBOARD_PORT && is_vnc_active()) {
+        vnc_clipboard_data_from_guest(buf, len);
+        return len;
+    }
     port = get_port_from_id(id);
     if (port && port->hd) {
         write_len = qemu_chr_write(port->hd, buf, len);
@@ -211,6 +216,16 @@ static void write_to_port(VirtIOSerialPort *port,
     virtio_notify(port->virtserial->vdev, vq);
 }
 
+void virtio_serial_send_clipboard_to_guest(const uint8_t *buf, size_t len)
+{
+    VirtIOSerialPort *port = get_port_from_id(VIRTIO_SERIAL_CLIPBOARD_PORT);
+
+    if (!port) {
+        return;
+    }
+    write_to_port(port, buf, len);
+}
+
 /* Readiness of the guest to accept data on a port */
 static int cons_can_read(void *opaque)
 {
diff --git a/hw/virtio-serial.h b/hw/virtio-serial.h
index 08c4b51..44238f6 100644
--- a/hw/virtio-serial.h
+++ b/hw/virtio-serial.h
@@ -18,6 +18,9 @@
 #define VIRTIO_ID_SERIAL		7
 #define VIRTIO_SERIAL_BAD_ID		(~(uint32_t)0)
 
+/* Port number to function mapping */
+#define VIRTIO_SERIAL_CLIPBOARD_PORT	3
+
 struct virtio_serial_config
 {
     uint32_t max_nr_ports;
@@ -32,5 +35,6 @@ struct virtio_serial_config
 void *virtio_serial_new_port(PCIDevice *dev, int idx);
 void virtio_serial_monitor_command(Monitor *mon,
                                    const char *command, const char *param);
+void virtio_serial_send_clipboard_to_guest(const uint8_t *buf, size_t len);
 
 #endif
diff --git a/vnc.c b/vnc.c
index e4e78dc..271b64e 100644
--- a/vnc.c
+++ b/vnc.c
@@ -29,6 +29,7 @@
 #include "qemu_socket.h"
 #include "qemu-timer.h"
 #include "acl.h"
+#include "hw/virtio-serial.h"
 
 #define VNC_REFRESH_INTERVAL (1000 / 30)
 
@@ -671,6 +672,28 @@ static void vnc_copy(VncState *vs, int src_x, int src_y, int dst_x, int dst_y, i
     vnc_flush(vs);
 }
 
+void vnc_clipboard_data_from_guest(const uint8_t *buf, size_t len)
+{
+    VncState *vs;
+    VncDisplay *vd;
+    DisplayState *ds;
+
+    ds = vnc_display->ds;
+    vd = ds->opaque;
+
+    for (vs = vd->clients; vs; vs = vs->next) {
+        vnc_write_u8(vs, 3);  /* ServerCutText */
+        vnc_write_u8(vs, 0);  /* Padding */
+        vnc_write_u8(vs, 0);  /* Padding */
+        vnc_write_u8(vs, 0);  /* Padding */
+        vnc_write_u32(vs, len);
+        while (len--) {
+            vnc_write_u8(vs, *(buf++));
+        }
+        vnc_flush(vs);
+    }
+}
+
 static void vnc_dpy_copy(DisplayState *ds, int src_x, int src_y, int dst_x, int dst_y, int w, int h)
 {
     VncDisplay *vd = ds->opaque;
@@ -1239,6 +1262,7 @@ uint32_t read_u32(uint8_t *data, size_t offset)
 
 static void client_cut_text(VncState *vs, size_t len, uint8_t *text)
 {
+    virtio_serial_send_clipboard_to_guest(text, len);
 }
 
 static void check_pointer_type_change(VncState *vs, int absolute)
diff --git a/vnc.h b/vnc.h
index 9739c35..00afdbb 100644
--- a/vnc.h
+++ b/vnc.h
@@ -303,6 +303,8 @@ int vnc_client_io_error(VncState *vs, int ret, int last_errno);
 void start_client_init(VncState *vs);
 void start_auth_vnc(VncState *vs);
 
+void vnc_clipboard_data_from_guest(const uint8_t *buf, size_t len);
+
 /* Buffer management */
 void buffer_reserve(Buffer *buffer, size_t len);
 int buffer_empty(Buffer *buffer);
-- 
1.6.2.5

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

* [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-27 18:04 [Qemu-devel] virtio-serial: An interface for host-guest communication Amit Shah
  2009-07-27 18:04 ` [Qemu-devel] [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Amit Shah
@ 2009-07-27 20:22 ` Anthony Liguori
  2009-07-27 20:32   ` Daniel P. Berrange
  1 sibling, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-07-27 20:22 UTC (permalink / raw)
  To: Amit Shah; +Cc: qemu-devel, kvm, virtualization

Amit Shah wrote:
> Hello all,
>
> This are the latest version of the patches.
>
> Lots of things have changed since the last submission. A few of
> which I remember:
> - VNC copy / paste works* (* conditions apply)
>   - client vnc copies get propagated to guest port 3 (/dev/vmch3)
>   - guest writes to port 3 (/dev/vmch3) go straight to client's clipboard
>   

Why 3?

Where's the guest application that drives the copy/paste?

I expect the first problem you'll run into is that copy/paste daemon has 
to run as an unprivileged user but /dev/vmch3 is going to be owned by 
root.  You could set udev rules for /dev/vmch3 but that's pretty 
terrible IMHO.

I think you'll find that you need a root daemon that talks to vmchannel 
and then allows unprivileged connections over a unix socket device.  In 
that case, why even bother having multiple channels since your daemon 
can multiplex..

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-27 20:22 ` [Qemu-devel] Re: virtio-serial: An interface for host-guest communication Anthony Liguori
@ 2009-07-27 20:32   ` Daniel P. Berrange
  2009-07-27 20:37     ` Anthony Liguori
  2009-07-27 20:46     ` Jamie Lokier
  0 siblings, 2 replies; 62+ messages in thread
From: Daniel P. Berrange @ 2009-07-27 20:32 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: Amit Shah, qemu-devel, kvm, virtualization

On Mon, Jul 27, 2009 at 03:22:54PM -0500, Anthony Liguori wrote:
> Amit Shah wrote:
> >Hello all,
> >
> >This are the latest version of the patches.
> >
> >Lots of things have changed since the last submission. A few of
> >which I remember:
> >- VNC copy / paste works* (* conditions apply)
> >  - client vnc copies get propagated to guest port 3 (/dev/vmch3)
> >  - guest writes to port 3 (/dev/vmch3) go straight to client's clipboard
> >  
> 
> Why 3?
> 
> Where's the guest application that drives the copy/paste?
> 
> I expect the first problem you'll run into is that copy/paste daemon has 
> to run as an unprivileged user but /dev/vmch3 is going to be owned by 
> root.  You could set udev rules for /dev/vmch3 but that's pretty 
> terrible IMHO.

I don't think that's not too bad, for example, with fast-user-switching 
between multiple X servers and/or text consoles, there's already support
code that deals with chown'ing things like /dev/snd/* devices to match 
the active console session. Doing the same with the /dev/vmch3 device so
that it is only ever accessible to the current logged in user actually
fits in to that scheme quite well.

Daniel
-- 
|: Red Hat, Engineering, London   -o-   http://people.redhat.com/berrange/ :|
|: http://libvirt.org  -o-  http://virt-manager.org  -o-  http://ovirt.org :|
|: http://autobuild.org       -o-         http://search.cpan.org/~danberr/ :|
|: GnuPG: 7D3B9505  -o-  F3C9 553F A1DA 4AC2 5648 23C1 B3DF F742 7D3B 9505 :|

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-27 20:32   ` Daniel P. Berrange
@ 2009-07-27 20:37     ` Anthony Liguori
  2009-07-27 20:46     ` Jamie Lokier
  1 sibling, 0 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-07-27 20:37 UTC (permalink / raw)
  To: Daniel P. Berrange; +Cc: Amit Shah, qemu-devel, kvm, virtualization

Daniel P. Berrange wrote:
> I don't think that's not too bad, for example, with fast-user-switching
> between multiple X servers and/or text consoles, there's already support
> code that deals with chown'ing things like /dev/snd/* devices to match 
> the active console session. Doing the same with the /dev/vmch3 device so
> that it is only ever accessible to the current logged in user actually
> fits in to that scheme quite well.
>   

Yeah, I'm not sure how something like this would interact with f-u-s.

If copy/paste daemon for user foo opens /dev/vmch3 directly, when you 
switch users, how do you forcefully disconnect user foo from /dev/vmch3 
so that user bad can start using it?

Regards,

Anthony Liguori

> Daniel
>   

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-27 20:32   ` Daniel P. Berrange
  2009-07-27 20:37     ` Anthony Liguori
@ 2009-07-27 20:46     ` Jamie Lokier
  2009-07-27 23:44       ` Anthony Liguori
  1 sibling, 1 reply; 62+ messages in thread
From: Jamie Lokier @ 2009-07-27 20:46 UTC (permalink / raw)
  To: Daniel P. Berrange; +Cc: Amit Shah, virtualization, qemu-devel, kvm

Daniel P. Berrange wrote:
> > I expect the first problem you'll run into is that copy/paste daemon has 
> > to run as an unprivileged user but /dev/vmch3 is going to be owned by 
> > root.  You could set udev rules for /dev/vmch3 but that's pretty 
> > terrible IMHO.
> 
> I don't think that's not too bad, for example, with fast-user-switching 
> between multiple X servers and/or text consoles, there's already support
> code that deals with chown'ing things like /dev/snd/* devices to match 
> the active console session. Doing the same with the /dev/vmch3 device so
> that it is only ever accessible to the current logged in user actually
> fits in to that scheme quite well.

With multiple X servers, there can be more than one currently logged in user.

Same with multiple text consoles - that's more familiar.

Which one owns /dev/vmch3?

-- Jamie

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-27 20:46     ` Jamie Lokier
@ 2009-07-27 23:44       ` Anthony Liguori
  2009-07-28 10:36         ` Amit Shah
       [not found]         ` <20090728140029.GA16067@amd.home.annexia.org>
  0 siblings, 2 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-07-27 23:44 UTC (permalink / raw)
  To: Jamie Lokier; +Cc: Amit Shah, qemu-devel, kvm, virtualization

Jamie Lokier wrote:
> With multiple X servers, there can be more than one currently logged in user.
>
> Same with multiple text consoles - that's more familiar.
>
> Which one owns /dev/vmch3?
>   

For a VMM, copy/paste should work with whatever user has the active X 
session that's controlling the physical display.

Yes, it could get complicated if we supported multiple video cards, but 
fortunately we don't :-)

I really think you need to have a copy/paste daemon that allows multiple 
X sessions to connect to it and then that daemon can somehow determine 
who is the "active" session.

This is part of the reason I've been pushing for a concrete example.  
All the signs here point to a privileged daemon that delegates to 
multiple users.  I think just about any use-case will have a similar model.

It really suggests that you need _one_ vmchannel that's exposed to 
userspace with a single userspace daemon that consumes it.  You want the 
flexibility of a userspace daemon in determining how you multiplex and 
do security.  I don't think it's something you want to bake into the 
userspace/kernel interface.

And if you have a single daemon that serves vmchannel sessions, that 
daemon can make it transparent whether the session is going over 
/dev/ttyS0, a network device, /dev/hvc1, etc.

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-27 23:44       ` Anthony Liguori
@ 2009-07-28 10:36         ` Amit Shah
       [not found]           ` <4A6F0048.1000103@codemonkey.ws>
       [not found]         ` <20090728140029.GA16067@amd.home.annexia.org>
  1 sibling, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-07-28 10:36 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: qemu-devel, kvm, virtualization

On (Mon) Jul 27 2009 [18:44:28], Anthony Liguori wrote:
> Jamie Lokier wrote:
>> With multiple X servers, there can be more than one currently logged in user.
>>
>> Same with multiple text consoles - that's more familiar.
>>
>> Which one owns /dev/vmch3?
>>   
>
> For a VMM, copy/paste should work with whatever user has the active X  
> session that's controlling the physical display.
>
> Yes, it could get complicated if we supported multiple video cards, but  
> fortunately we don't :-)
>
> I really think you need to have a copy/paste daemon that allows multiple  
> X sessions to connect to it and then that daemon can somehow determine  
> who is the "active" session.
>
> This is part of the reason I've been pushing for a concrete example.   
> All the signs here point to a privileged daemon that delegates to  
> multiple users.  I think just about any use-case will have a similar 
> model.
>
> It really suggests that you need _one_ vmchannel that's exposed to  
> userspace with a single userspace daemon that consumes it.  You want the  
> flexibility of a userspace daemon in determining how you multiplex and  
> do security.  I don't think it's something you want to bake into the  
> userspace/kernel interface.

Right; use virtio just as the transport and all the interesting
activity happens in userspaces. That was the basis with which I started.
I can imagine dbus doing the copy/paste, lock screen, etc. actions.

However for libguestfs, dbus isn't an option and they already have some
predefined agents for each port. So libguestfs is an example for a
multi-port usecase for virtio-serial.

> And if you have a single daemon that serves vmchannel sessions, that  
> daemon can make it transparent whether the session is going over  
> /dev/ttyS0, a network device, /dev/hvc1, etc.

or /dev/vmch0. it doesn't matter. All minimal virtio devices will look
the same. Pop buffers, populate them, push them, etc.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
       [not found]         ` <20090728140029.GA16067@amd.home.annexia.org>
@ 2009-07-28 14:48           ` Anthony Liguori
  2009-07-28 14:55             ` Richard W.M. Jones
  2009-08-03 19:57           ` Anthony Liguori
  1 sibling, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-07-28 14:48 UTC (permalink / raw)
  To: Richard W.M. Jones; +Cc: Amit Shah, qemu-devel, kvm, virtualization

Richard W.M. Jones wrote:
> On Mon, Jul 27, 2009 at 06:44:28PM -0500, Anthony Liguori wrote:
>   
>> It really suggests that you need _one_ vmchannel that's exposed to  
>> userspace with a single userspace daemon that consumes it.
>>     
>
> ... or a more flexible API.  I don't like having fixed /dev/vmch*
> devices either.
>   

Indeed.

> A long time ago (on a mailing list not so far away) there was a much
> better userspace API proposed, which had a separate AF_VMCHANNEL
> address family.
>
> That API works much more like TCP sockets, except without requiring
> network devices:
>   

Dave Miller nacked that approach with a sledgehammer instead preferring 
that we just use standard TCP/IP which is what led to the current 
implementation using slirp.

A userspace daemon with unix domain sockets could give a similar solution.

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-28 14:48           ` Anthony Liguori
@ 2009-07-28 14:55             ` Richard W.M. Jones
  2009-07-28 15:00               ` Anthony Liguori
  0 siblings, 1 reply; 62+ messages in thread
From: Richard W.M. Jones @ 2009-07-28 14:55 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: Amit Shah, qemu-devel, kvm, virtualization

On Tue, Jul 28, 2009 at 09:48:00AM -0500, Anthony Liguori wrote:
> Richard W.M. Jones wrote:
>> On Mon, Jul 27, 2009 at 06:44:28PM -0500, Anthony Liguori wrote:
>>   
>>> It really suggests that you need _one_ vmchannel that's exposed to   
>>> userspace with a single userspace daemon that consumes it.
>>>     
>>
>> ... or a more flexible API.  I don't like having fixed /dev/vmch*
>> devices either.
>>   
>
> Indeed.
>
>> A long time ago (on a mailing list not so far away) there was a much
>> better userspace API proposed, which had a separate AF_VMCHANNEL
>> address family.
>>
>> That API works much more like TCP sockets, except without requiring
>> network devices:
>>   
>
> Dave Miller nacked that approach with a sledgehammer instead preferring  
> that we just use standard TCP/IP which is what led to the current  
> implementation using slirp.

I'm aware of that - I just don't think it was a good choice.

[BTW the qemu-devel mailing list seems to be bouncing messages]

Rich.

-- 
Richard Jones, Emerging Technologies, Red Hat  http://et.redhat.com/~rjones
libguestfs lets you edit virtual machines.  Supports shell scripting,
bindings from many languages.  http://et.redhat.com/~rjones/libguestfs/
See what it can do: http://et.redhat.com/~rjones/libguestfs/recipes.html

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-28 14:55             ` Richard W.M. Jones
@ 2009-07-28 15:00               ` Anthony Liguori
  0 siblings, 0 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-07-28 15:00 UTC (permalink / raw)
  To: Richard W.M. Jones; +Cc: Amit Shah, qemu-devel, kvm, virtualization

Richard W.M. Jones wrote:
> On Tue, Jul 28, 2009 at 09:48:00AM -0500, Anthony Liguori wrote:
>   
>> Dave Miller nacked that approach with a sledgehammer instead preferring  
>> that we just use standard TCP/IP which is what led to the current  
>> implementation using slirp.
>>     
>
> I'm aware of that - I just don't think it was a good choice.
>
> [BTW the qemu-devel mailing list seems to be bouncing messages]
>   

I know.  I've reported it to the Savannah admins and am helping them 
track it down.

> Rich.
>
>   

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
       [not found]           ` <4A6F0048.1000103@codemonkey.ws>
@ 2009-07-29  7:44             ` Amit Shah
  2009-07-29  7:48               ` Gleb Natapov
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-07-29  7:44 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: qemu-devel, kvm, virtualization

On (Tue) Jul 28 2009 [08:42:32], Anthony Liguori wrote:
> Amit Shah wrote:
>> Right; use virtio just as the transport and all the interesting
>> activity happens in userspaces. That was the basis with which I started.
>> I can imagine dbus doing the copy/paste, lock screen, etc. actions.
>>
>> However for libguestfs, dbus isn't an option and they already have some
>> predefined agents for each port. So libguestfs is an example for a
>> multi-port usecase for virtio-serial.
>>   
>
> Or don't use dbus and use something that libguestfs is able to embed.   
> The fact that libguestfs doesn't want dbus in the guest is not an  
> argument for using a higher level kernel interface especially one that  
> doesn't meet the requirements of the interface.

But why do we want to limit the device to only one port? It's not too
complex supporting additional ones.

As I see it qemu and the kernel should provide the basic abstraction for
the userspace to go do its job. Why create unnecessary barriers?

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-29  7:44             ` Amit Shah
@ 2009-07-29  7:48               ` Gleb Natapov
  2009-08-05 18:00                 ` Jamie Lokier
  0 siblings, 1 reply; 62+ messages in thread
From: Gleb Natapov @ 2009-07-29  7:48 UTC (permalink / raw)
  To: Amit Shah; +Cc: kvm, qemu-devel, virtualization

On Wed, Jul 29, 2009 at 01:14:18PM +0530, Amit Shah wrote:
> On (Tue) Jul 28 2009 [08:42:32], Anthony Liguori wrote:
> > Amit Shah wrote:
> >> Right; use virtio just as the transport and all the interesting
> >> activity happens in userspaces. That was the basis with which I started.
> >> I can imagine dbus doing the copy/paste, lock screen, etc. actions.
> >>
> >> However for libguestfs, dbus isn't an option and they already have some
> >> predefined agents for each port. So libguestfs is an example for a
> >> multi-port usecase for virtio-serial.
> >>   
> >
> > Or don't use dbus and use something that libguestfs is able to embed.   
> > The fact that libguestfs doesn't want dbus in the guest is not an  
> > argument for using a higher level kernel interface especially one that  
> > doesn't meet the requirements of the interface.
> 
> But why do we want to limit the device to only one port? It's not too
> complex supporting additional ones.
> 
> As I see it qemu and the kernel should provide the basic abstraction for
> the userspace to go do its job. Why create unnecessary barriers?
> 
I agree. If userspace wants it may use only one channel and demultiplex
messages by itself, but we shouldn't force it to. Also one of the
requirements for virtio-serial is to have connect disconnect
notifications. It is not possible with demultiplexing in the userspace.

--
			Gleb.

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
       [not found]         ` <20090728140029.GA16067@amd.home.annexia.org>
  2009-07-28 14:48           ` Anthony Liguori
@ 2009-08-03 19:57           ` Anthony Liguori
  2009-08-05 17:57             ` Jamie Lokier
  2009-08-05 18:32             ` Richard W.M. Jones
  1 sibling, 2 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-08-03 19:57 UTC (permalink / raw)
  To: Richard W.M. Jones; +Cc: Amit Shah, qemu-devel, kvm, virtualization

Richard W.M. Jones wrote:
> On Mon, Jul 27, 2009 at 06:44:28PM -0500, Anthony Liguori wrote:
>   
>> It really suggests that you need _one_ vmchannel that's exposed to  
>> userspace with a single userspace daemon that consumes it.
>>     
>
> ... or a more flexible API.  I don't like having fixed /dev/vmch*
> devices either.
>   

Have you considered using a usb serial device?  Something attractive 
about it is that a productid/vendorid can be specified which means that 
you can use that as a method of enumerating devices.

Hot add/remove is supported automagically.

Regards,

Anthony Liguori

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

* [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication
  2009-07-27 18:04 ` [Qemu-devel] [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Amit Shah
  2009-07-27 18:04   ` [Qemu-devel] [PATCH 1/3] virtio-serial: virtio device for simple host <-> guest communication Amit Shah
@ 2009-08-05  0:03   ` Rusty Russell
  2009-08-05  5:12     ` Amit Shah
  2009-08-05  9:58     ` Amit Shah
  1 sibling, 2 replies; 62+ messages in thread
From: Rusty Russell @ 2009-08-05  0:03 UTC (permalink / raw)
  To: virtualization; +Cc: Amit Shah, qemu-devel, kvm

On Tue, 28 Jul 2009 03:34:33 am Amit Shah wrote:
> We expose multiple char devices ("ports") for simple communication
> between the host userspace and guest.

Hi Amit,

   OK, seems like it's time for some serious review.  Below.

> +config VIRTIO_SERIAL
> +	tristate "Virtio serial"
> +	select VIRTIO
> +	select VIRTIO_RING
> +	help
> +	  Virtio serial device driver for simple guest and host communication

depends on VIRTIO
Do not "select VIRTIO_RING" -- this code doesn't explicitly rely on it.

> +struct virtio_serial_struct {
> +	struct work_struct rx_work;
> +	struct work_struct tx_work;
> +	struct work_struct queue_work;
> +	struct work_struct config_work;
> +
> +	struct list_head port_head;
> +
> +	struct virtio_device *vdev;
> +	struct class *class;
> +	struct virtqueue *in_vq, *out_vq;
> +
> +	struct virtio_serial_config *config;
> +};
> +
> +/* This struct holds individual buffers received for each port */
> +struct virtio_serial_port_buffer {
> +	struct list_head list;
> +
> +	unsigned int len; /* length of the buffer */
> +	unsigned int offset; /* offset in the buf from which to consume data */
> +
> +	char *buf;
> +};
> +
> +/* This struct is put in each buffer that gets passed to userspace and
> + * vice-versa
> + */
> +struct virtio_serial_id {
> +	u32 id; /* Port number */
> +};
> +
> +struct virtio_serial_port {
> +	/* Next port in the list */
> +	struct list_head next;
> +
> +	/* Buffer management */
> +	struct virtio_serial_port_buffer read_buf;
> +	struct list_head readbuf_head;
> +	struct completion have_data;
> +
> +	/* Each port associates with a separate char device */
> +	struct cdev cdev;
> +	struct device *dev;
> +};
> +
> +static struct virtio_serial_struct virtserial;
> +
> +static int major = 60; /* from the experimental range */

This will obviously need to change before it goes in.

> +
> +static struct virtio_serial_port *get_port_from_id(u32 id)
> +{
> +	struct virtio_serial_port *port;
> +	struct list_head *ptr;
> +
> +	list_for_each(ptr, &virtserial.port_head) {
> +		port = list_entry(ptr, struct virtio_serial_port, next);

list_for_each_entry, same with the others.

> +static int get_id_from_port(struct virtio_serial_port *port)
> +{
> +	struct virtio_serial_port *match;
> +	struct list_head *ptr;
> +
> +	list_for_each(ptr, &virtserial.port_head) {
> +		match = list_entry(ptr, struct virtio_serial_port, next);
> +
> +		if (match == port)
> +			return MINOR(port->dev->devt);
> +	}
> +	return VIRTIO_SERIAL_BAD_ID;
> +}

Why does this exist?  Seems weird to loop, given you have the pointer already?

> +
> +static struct virtio_serial_port *get_port_from_buf(char *buf)
> +{
> +	u32 id;
> +
> +	memcpy(&id, buf, sizeof(id));
> +
> +	return get_port_from_id(id);
> +}
> +
> +
> +static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
> +			       size_t count, loff_t *offp)
> +{
> +	struct list_head *ptr, *ptr2;
> +	struct virtio_serial_port *port;
> +	struct virtio_serial_port_buffer *buf;
> +	ssize_t ret;
> +
> +	port = filp->private_data;
> +
> +	ret = -EINTR;
> +	if (list_empty(&port->readbuf_head)) {
> +		if (filp->f_flags & O_NONBLOCK)
> +			return -EAGAIN;
> +
> +		if (wait_for_completion_interruptible(&port->have_data) < 0)
> +			return ret;
> +	}

I don't think this code works. What protects from two simultaneous readers?
Who resets the completion? It's more normal to use a waitqueue and
wait_event_interruptible().

> +	list_for_each_safe(ptr, ptr2, &port->readbuf_head) {
> +		buf = list_entry(ptr, struct virtio_serial_port_buffer, list);
> +
> +		/* FIXME: other buffers further in this list might
> +		 * have data too
> +		 */
> +		if (count > buf->len - buf->offset)
> +			count = buf->len - buf->offset;
> +
> +		ret = copy_to_user(ubuf, buf->buf + buf->offset, count);
> +
> +		/* Return the number of bytes actually copied */
> +		ret = count - ret;
> +
> +		buf->offset += ret;
> +
> +		if (buf->len - buf->offset == 0) {
> +			list_del(&buf->list);
> +			kfree(buf->buf);
> +			kfree(buf);
> +		}
> +		/* FIXME: if there's more data requested and more data
> +		 * available, return it.
> +		 */
> +		break;

There's nothing wrong with short reads here.

> +	}
> +	return ret;

This can return -EINTR; if you hadn't assigned ret = -EINTR unconditionally
above, gcc would have given you a warning about that path :)

> +}
> +
> +/* For data that exceeds PAGE_SIZE in size we should send it all in
> + * one sg to not unnecessarily split up the data. Also some (all?)
> + * vnc clients don't consume split data.
> + *
> + * If we are to keep PAGE_SIZE sized buffers, we then have to stack
> + * multiple of those in one virtio request. virtio-ring returns to us
> + * just one pointer for all the buffers. So use this struct to
> + * allocate the bufs in so that freeing this up later is easier.
> + */
> +struct vbuf {
> +	char **bufs;
> +	struct scatterlist *sg;
> +	unsigned int nent;
> +};
> +
> +static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
> +				size_t count, loff_t *offp)
> +{
> +	struct virtqueue *out_vq;
> +	struct virtio_serial_port *port;
> +	struct virtio_serial_id id;
> +	struct vbuf *vbuf;
> +	size_t offset, size;
> +	ssize_t ret;
> +	int i, id_len;
> +
> +	port = filp->private_data;
> +	id.id = get_id_from_port(port);
> +	out_vq = virtserial.out_vq;
> +
> +	id_len = sizeof(id);
> +
> +	ret = -EFBIG;
> +	vbuf = kzalloc(sizeof(struct vbuf), GFP_KERNEL);
> +	if (!vbuf)
> +		return ret;

-ENOMEM is normal here.

> +
> +	/* Max. number of buffers clubbed together in one message */
> +	vbuf->nent = (count + id_len + PAGE_SIZE - 1) / PAGE_SIZE;
> +
> +	vbuf->bufs = kzalloc(vbuf->nent, GFP_KERNEL);
> +	if (!vbuf->bufs)
> +		goto free_vbuf;
> +
> +	vbuf->sg = kzalloc(vbuf->nent, GFP_KERNEL);
> +	if (!vbuf->sg)
> +		goto free_bufs;
> +	sg_init_table(vbuf->sg, vbuf->nent);
> +
> +	i = 0; /* vbuf->bufs[i] */
> +	offset = 0; /* offset in the user buffer */
> +	while (count - offset) {
> +		size = min(count - offset + id_len, PAGE_SIZE);
> +		vbuf->bufs[i] = kzalloc(size, GFP_KERNEL);
> +		if (!vbuf->bufs[i]) {
> +			ret = -EFBIG;
> +			goto free_buffers;
> +		}
> +		if (id_len) {
> +			memcpy(vbuf->bufs[i], &id, id_len);
> +			size -= id_len;
> +		}
> +		ret = copy_from_user(vbuf->bufs[i] + id_len, ubuf + offset, size);
> +		offset += size - ret;
> +
> +		sg_set_buf(&vbuf->sg[i], vbuf->bufs[i], size - ret + id_len);
> +		id_len = 0; /* Pass the port id only in the first buffer */
> +		i++;
> +	}
> +	if (out_vq->vq_ops->add_buf(out_vq, vbuf->sg, i, 0, vbuf)) {
> +		/* XXX: We can't send the buffer. Report failure */
> +		ret = 0;
> +	}
> +	/* Tell Host to go! */
> +	out_vq->vq_ops->kick(out_vq);
> +
> +	/* We're expected to return the amount of data we wrote */
> +	return offset;
> +free_buffers:
> +	while (i--)
> +		kfree(vbuf->bufs[i]);
> +	kfree(vbuf->sg);
> +free_bufs:
> +	kfree(vbuf->bufs);
> +free_vbuf:
> +	kfree(vbuf);
> +	return ret;
> +}
> +
> +static long virtserial_ioctl(struct file *filp, unsigned int ioctl,
> +			     unsigned long arg)
> +{
> +	struct virtio_serial_port *port;
> +	long ret;
> +
> +	port = filp->private_data;
> +
> +	ret = -EINVAL;
> +	switch (ioctl) {
> +	default:
> +		break;
> +	}
> +	return ret;
> +}

I thought -ENOTTY was normal for invalid ioctls? In which case, just don't
implement this function.

> +
> +static int virtserial_release(struct inode *inode, struct file *filp)
> +{
> +	filp->private_data = NULL;
> +	return 0;
> +}

This seems redundant.

> +
> +static int virtserial_open(struct inode *inode, struct file *filp)
> +{
> +	struct cdev *cdev = inode->i_cdev;
> +	struct virtio_serial_port *port;
> +
> +	port = container_of(cdev, struct virtio_serial_port, cdev);
> +
> +	filp->private_data = port;
> +	return 0;
> +}
> +
> +static unsigned int virtserial_poll(struct file *filp, poll_table *wait)
> +{
> +	pr_notice("%s\n", __func__);
> +	return 0;
> +}

And you're going to want to implement this, too.

> +
> +static const struct file_operations virtserial_fops = {
> +	.owner = THIS_MODULE,
> +	.open  = virtserial_open,
> +	.read  = virtserial_read,
> +	.write = virtserial_write,
> +	.compat_ioctl = virtserial_ioctl,
> +	.unlocked_ioctl = virtserial_ioctl,
> +	.poll  = virtserial_poll,
> +	.release = virtserial_release,
> +};
> +
> +static void virtio_serial_queue_work_handler(struct work_struct *work)
> +{
> +	struct scatterlist sg[1];
> +	struct virtqueue *vq;
> +	char *buf;
> +
> +	vq = virtserial.in_vq;
> +	while (1) {
> +		buf = kzalloc(PAGE_SIZE, GFP_KERNEL);
> +		if (!buf)
> +			break;
> +
> +		sg_init_one(sg, buf, PAGE_SIZE);
> +
> +		if (vq->vq_ops->add_buf(vq, sg, 0, 1, buf) < 0) {
> +			kfree(buf);
> +			break;
> +		}
> +	}
> +	vq->vq_ops->kick(vq);
> +}
> +
> +static void virtio_serial_rx_work_handler(struct work_struct *work)
> +{
> +	struct virtio_serial_port *port = NULL;
> +	struct virtio_serial_port_buffer *buf;
> +	struct virtqueue *vq;
> +	char *tmpbuf;
> +	unsigned int tmplen;
> +
> +	vq = virtserial.in_vq;
> +	while ((tmpbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
> +		port = get_port_from_buf(tmpbuf);
> +		if (!port) {
> +			/* No valid index at start of
> +			 * buffer. Drop it.
> +			 */
> +			pr_debug("%s: invalid index in buffer, %c %d\n",
> +				 __func__, tmpbuf[0], tmpbuf[0]);
> +			break;

leak?

> +		}
> +		buf = kzalloc(sizeof(struct virtio_serial_port_buffer),
> +			      GFP_KERNEL);
> +		if (!buf)
> +			break;
> +
> +		buf->buf = tmpbuf;
> +		buf->len = tmplen;
> +		buf->offset = sizeof(struct virtio_serial_id);
> +		list_add_tail(&buf->list, &port->readbuf_head);
> +
> +		complete(&port->have_data);
> +	}
> +	/* Allocate buffers for all the ones that got used up */
> +	schedule_work(&virtserial.queue_work);
> +}

Why do the allocation in a separate workqueue?

> +
> +static void virtio_serial_tx_work_handler(struct work_struct *work)
> +{
> +	struct virtqueue *vq;
> +	struct vbuf *vbuf;
> +	unsigned int tmplen;
> +	int i;
> +
> +	vq = virtserial.out_vq;
> +	while ((vbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
> +		for (i = 0; i < vbuf->nent; i++) {
> +			kfree(vbuf->bufs[i]);
> +		}
> +		kfree(vbuf->bufs);
> +		kfree(vbuf->sg);
> +		kfree(vbuf);
> +	}
> +}
> +
> +static void rx_intr(struct virtqueue *vq)
> +{
> +	schedule_work(&virtserial.rx_work);
> +}
> +
> +static void tx_intr(struct virtqueue *vq)
> +{
> +	schedule_work(&virtserial.tx_work);
> +}
> +
> +static void config_intr(struct virtio_device *vdev)
> +{
> +	schedule_work(&virtserial.config_work);
> +}
> +
> +static u32 virtserial_get_hot_add_port(struct virtio_serial_config *config)
> +{
> +	u32 i;
> +	u32 port_nr;
> +
> +	for (i = 0; i < virtserial.config->max_nr_ports / 32; i++) {
> +		port_nr = ffs(config->ports_map[i] ^ virtserial.config->ports_map[i]);
> +		if (port_nr)
> +			break;
> +	}
> +	if (unlikely(!port_nr))
> +		return VIRTIO_SERIAL_BAD_ID;
> +
> +	/* We used ffs above */
> +	port_nr--;
> +
> +	/* FIXME: Do this only when add_port is successful */
> +	virtserial.config->ports_map[i] |= 1U << port_nr;
> +
> +	port_nr += i * 32;
> +	return port_nr;
> +}
> +
> +static u32 virtserial_find_next_port(u32 *map, int *map_i)
> +{
> +	u32 port_nr;
> +
> +	while (1) {
> +		port_nr = ffs(*map);
> +		if (port_nr)
> +			break;
> +
> +		if (unlikely(*map_i >= virtserial.config->max_nr_ports / 32))
> +			return VIRTIO_SERIAL_BAD_ID;
> +		++*map_i;
> +		*map = virtserial.config->ports_map[*map_i];
> +	}
> +	/* We used ffs above */
> +	port_nr--;
> +
> +	/* FIXME: Do this only when add_port is successful / reset bit
> +	 * in config space if add_port was unsuccessful
> +	 */
> +	*map &= ~(1U << port_nr);
> +
> +	port_nr += *map_i * 32;
> +	return port_nr;
> +}
> +
> +static int virtserial_add_port(u32 port_nr)
> +{
> +	struct virtio_serial_port *port;
> +	dev_t devt;
> +	int ret;
> +
> +	port = kzalloc(sizeof(struct virtio_serial_port), GFP_KERNEL);
> +	if (!port)
> +		return -ENOMEM;
> +
> +	devt = MKDEV(major, port_nr);
> +	cdev_init(&port->cdev, &virtserial_fops);
> +
> +	ret = register_chrdev_region(devt, 1, "virtio-serial");
> +	if (ret < 0) {
> +		pr_err("%s: error registering chrdev region, ret = %d\n",
> +		       __func__, ret);
> +		goto free_cdev;
> +	}
> +	ret = cdev_add(&port->cdev, devt, 1);
> +	if (ret < 0) {
> +		pr_err("%s: error adding cdev, ret = %d\n", __func__, ret);
> +		goto free_cdev;
> +	}
> +	port->dev = device_create(virtserial.class, NULL, devt, NULL,
> +				  "vmch%u", port_nr);
> +	if (IS_ERR(port->dev)) {
> +		ret = PTR_ERR(port->dev);
> +		pr_err("%s: Error creating device, ret = %d\n", __func__, ret);
> +		goto free_cdev;
> +	}
> +	INIT_LIST_HEAD(&port->readbuf_head);
> +	init_completion(&port->have_data);
> +
> +	list_add_tail(&port->next, &virtserial.port_head);
> +
> +	pr_info("virtio-serial port found at id %u\n", port_nr);
> +
> +	return 0;
> +free_cdev:
> +	unregister_chrdev(major, "virtio-serial");
> +	return ret;
> +}
> +
> +static __u32 get_ports_map_size(__u32 max_ports)
> +{
> +	return sizeof(__u32) * ((max_ports + 31) / 32);
> +}

The __ versions are for user-visible headers only.

> +
> +static void virtio_serial_config_work_handler(struct work_struct *work)
> +{
> +	struct virtio_serial_config *virtserconf;
> +	struct virtio_device *vdev = virtserial.vdev;
> +	u32 i, port_nr;
> +	int ret;
> +
> +	virtserconf = kmalloc(sizeof(struct virtio_serial_config) +
> +			      get_ports_map_size(virtserial.config->max_nr_ports),
> +			      GFP_KERNEL);
> +	vdev->config->get(vdev,
> +			  offsetof(struct virtio_serial_config, nr_active_ports),
> +			  &virtserconf->nr_active_ports,
> +			  sizeof(virtserconf->nr_active_ports));
> +	vdev->config->get(vdev,
> +			  offsetof(struct virtio_serial_config, ports_map),
> +			  virtserconf->ports_map,
> +			  get_ports_map_size(virtserial.config->max_nr_ports));
> +
> +	/* Hot-add ports */
> +	for (i = virtserial.config->nr_active_ports;
> +	     i < virtserconf->nr_active_ports; i++) {
> +		port_nr = virtserial_get_hot_add_port(virtserconf);
> +		if (port_nr == VIRTIO_SERIAL_BAD_ID)
> +			continue;
> +		ret = virtserial_add_port(port_nr);
> +		if (!ret)
> +			virtserial.config->nr_active_ports++;
> +	}
> +	kfree(virtserconf);
> +}
> +
> +static int virtserial_probe(struct virtio_device *vdev)
> +{
> +	struct virtqueue *vqs[3];

3?

> +	const char *vq_names[] = { "input", "output" };
> +	vq_callback_t *vq_callbacks[] = { rx_intr, tx_intr };
> +	u32 i, map;
> +	int ret, map_i;
> +	u32 max_nr_ports;
> +
> +	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
> +					 max_nr_ports),
> +			  &max_nr_ports,
> +			  sizeof(max_nr_ports));
> +	virtserial.config = kmalloc(sizeof(struct virtio_serial_config)
> +				    + get_ports_map_size(max_nr_ports),
> +				    GFP_KERNEL);

kmalloc not checked.

> +	virtserial.config->max_nr_ports = max_nr_ports;
> +
> +	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
> +					 nr_active_ports),
> +			  &virtserial.config->nr_active_ports,
> +			  sizeof(virtserial.config->nr_active_ports));
> +	vdev->config->get(vdev,
> +			  offsetof(struct virtio_serial_config, ports_map),
> +			  virtserial.config->ports_map,
> +			  get_ports_map_size(max_nr_ports));
> +
> +	virtserial.vdev = vdev;
> +
> +	ret = vdev->config->find_vqs(vdev, 2, vqs, vq_callbacks, vq_names);
> +	if (ret)
> +		goto fail;
> +
> +	virtserial.in_vq = vqs[0];
> +	virtserial.out_vq = vqs[1];
> +
> +	INIT_LIST_HEAD(&virtserial.port_head);
> +
> +	map_i = 0;
> +	map = virtserial.config->ports_map[map_i];
> +	for (i = 0; i < virtserial.config->nr_active_ports; i++) {
> +		__u32 port_nr;
> +
> +		port_nr = virtserial_find_next_port(&map, &map_i);
> +		if (unlikely(port_nr == VIRTIO_SERIAL_BAD_ID))
> +			continue;
> +
> +		virtserial_add_port(port_nr);
> +	}
> +	INIT_WORK(&virtserial.rx_work, &virtio_serial_rx_work_handler);
> +	INIT_WORK(&virtserial.tx_work, &virtio_serial_tx_work_handler);
> +	INIT_WORK(&virtserial.queue_work, &virtio_serial_queue_work_handler);
> +	INIT_WORK(&virtserial.config_work, &virtio_serial_config_work_handler);
> +
> +	/* Allocate pages to fill the receive queue */
> +	schedule_work(&virtserial.queue_work);
> +
> +	return 0;
> +fail:
> +	return ret;
> +}
> +
> +
> +static void virtserial_remove_port_data(struct virtio_serial_port *port)
> +{
> +	struct list_head *ptr, *ptr2;
> +
> +	device_destroy(virtserial.class, port->dev->devt);
> +	unregister_chrdev_region(port->dev->devt, 1);
> +	cdev_del(&port->cdev);
> +
> +	/* Remove the buffers in which we have unconsumed data */
> +	list_for_each_safe(ptr, ptr2, &port->readbuf_head) {
> +		struct virtio_serial_port_buffer *buf;
> +
> +		buf = list_entry(ptr, struct virtio_serial_port_buffer, list);
> +
> +		list_del(&buf->list);
> +		kfree(buf->buf);
> +		kfree(buf);
> +	}
> +}
> +
> +static void virtserial_remove(struct virtio_device *vdev)
> +{
> +	struct list_head *ptr, *ptr2;
> +	char *buf;
> +	int len;
> +
> +	unregister_chrdev(major, "virtio-serial");
> +	class_destroy(virtserial.class);
> +
> +	cancel_work_sync(&virtserial.rx_work);
> +
> +	/* Free up the unused buffers in the receive queue */
> +	while ((buf = virtserial.in_vq->vq_ops->get_buf(virtserial.in_vq, &len)))
> +		kfree(buf);

This won't quite work.  get_buf gets *used* buffers.  You need to track
buffers yourself and delete them after del_vqs.

> +	vdev->config->del_vqs(vdev);
> +
> +	list_for_each_safe(ptr, ptr2, &virtserial.port_head) {
> +		struct virtio_serial_port *port;
> +
> +		port = list_entry(ptr, struct virtio_serial_port, next);
> +
> +		list_del(&port->next);
> +		virtserial_remove_port_data(port);
> +		kfree(port);
> +	}
> +	kfree(virtserial.config);
> +}
> +
> +static struct virtio_device_id id_table[] = {
> +	{ VIRTIO_ID_SERIAL, VIRTIO_DEV_ANY_ID },
> +	{ 0 },
> +};
> +
> +static struct virtio_driver virtio_serial = {
> +  //	.feature_table = features,
> +  //	.feature_table_size = ARRAY_SIZE(features),
> +	.driver.name =	KBUILD_MODNAME,
> +	.driver.owner =	THIS_MODULE,
> +	.id_table =	id_table,
> +	.probe =	virtserial_probe,
> +	.remove =	virtserial_remove,
> +	.config_changed = config_intr,
> +};
> +
> +static int __init init(void)
> +{
> +	int ret;
> +
> +	virtserial.class = class_create(THIS_MODULE, "virtio-serial");
> +	if (IS_ERR(virtserial.class)) {
> +		pr_err("Error creating virtio-serial class\n");
> +		ret = PTR_ERR(virtserial.class);
> +		return ret;
> +	}
> +	ret = register_virtio_driver(&virtio_serial);
> +	if (ret) {
> +		class_destroy(virtserial.class);
> +		return ret;
> +	}
> +	return 0;
> +}
> +
> +static void __exit fini(void)
> +{
> +	unregister_virtio_driver(&virtio_serial);
> +}
> +module_init(init);
> +module_exit(fini);
> +
> +MODULE_DEVICE_TABLE(virtio, id_table);
> +MODULE_DESCRIPTION("Virtio serial driver");
> +MODULE_LICENSE("GPL");
> diff --git a/include/linux/virtio_serial.h b/include/linux/virtio_serial.h
> new file mode 100644
> index 0000000..025dcf1
> --- /dev/null
> +++ b/include/linux/virtio_serial.h
> @@ -0,0 +1,27 @@
> +#ifndef _LINUX_VIRTIO_SERIAL_H
> +#define _LINUX_VIRTIO_SERIAL_H
> +#include <linux/types.h>
> +#include <linux/virtio_config.h>
> +
> +/* Guest kernel - Host interface */
> +
> +/* The ID for virtio serial */
> +#define VIRTIO_ID_SERIAL		7
> +
> +#define VIRTIO_SERIAL_BAD_ID		(~(u32)0)
> +
> +struct virtio_serial_config {
> +	__u32 max_nr_ports;
> +	__u32 nr_active_ports;
> +	__u32 ports_map[0 /* (max_nr_ports + 31) / 32 */];
> +};
> +
> +#ifdef __KERNEL__
> +
> +/* Guest kernel - Guest userspace interface */
> +
> +/* IOCTL-related */
> +#define VIRTIO_SERIAL_IO 0xAF

??

Cheers,
Rusty.

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

* [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication
  2009-08-05  0:03   ` [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Rusty Russell
@ 2009-08-05  5:12     ` Amit Shah
  2009-08-05  9:58     ` Amit Shah
  1 sibling, 0 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-05  5:12 UTC (permalink / raw)
  To: Rusty Russell; +Cc: qemu-devel, kvm, virtualization

On (Wed) Aug 05 2009 [09:33:40], Rusty Russell wrote:
> On Tue, 28 Jul 2009 03:34:33 am Amit Shah wrote:
> > We expose multiple char devices ("ports") for simple communication
> > between the host userspace and guest.
> 
> Hi Amit,
> 
>    OK, seems like it's time for some serious review.  Below.

Thanks! :-)

> > +config VIRTIO_SERIAL
> > +	tristate "Virtio serial"
> > +	select VIRTIO
> > +	select VIRTIO_RING
> > +	help
> > +	  Virtio serial device driver for simple guest and host communication
> 
> depends on VIRTIO
> Do not "select VIRTIO_RING" -- this code doesn't explicitly rely on it.

OK

> > +static int major = 60; /* from the experimental range */
> 
> This will obviously need to change before it goes in.

Yes. Once everything's OK'ed I'll CC Alan.

> > +
> > +static struct virtio_serial_port *get_port_from_id(u32 id)
> > +{
> > +	struct virtio_serial_port *port;
> > +	struct list_head *ptr;
> > +
> > +	list_for_each(ptr, &virtserial.port_head) {
> > +		port = list_entry(ptr, struct virtio_serial_port, next);
> 
> list_for_each_entry, same with the others.

OK

> > +static int get_id_from_port(struct virtio_serial_port *port)
> > +{
> > +	struct virtio_serial_port *match;
> > +	struct list_head *ptr;
> > +
> > +	list_for_each(ptr, &virtserial.port_head) {
> > +		match = list_entry(ptr, struct virtio_serial_port, next);
> > +
> > +		if (match == port)
> > +			return MINOR(port->dev->devt);
> > +	}
> > +	return VIRTIO_SERIAL_BAD_ID;
> > +}
> 
> Why does this exist?  Seems weird to loop, given you have the pointer already?

Hm, yes. Remnant from older code which used an array for storing port
structures. So the id was found by iterating and returning the index
where it was found. While converting the array to a linked list I just
changed this mechanically. Thanks for spotting it!

> > +static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
> > +			       size_t count, loff_t *offp)
> > +{
> > +	struct list_head *ptr, *ptr2;
> > +	struct virtio_serial_port *port;
> > +	struct virtio_serial_port_buffer *buf;
> > +	ssize_t ret;
> > +
> > +	port = filp->private_data;
> > +
> > +	ret = -EINTR;
> > +	if (list_empty(&port->readbuf_head)) {
> > +		if (filp->f_flags & O_NONBLOCK)
> > +			return -EAGAIN;
> > +
> > +		if (wait_for_completion_interruptible(&port->have_data) < 0)
> > +			return ret;
> > +	}
> 
> I don't think this code works. What protects from two simultaneous readers?
> Who resets the completion? It's more normal to use a waitqueue and
> wait_event_interruptible().

Yes, I'm going to switch this now that I have to use waitqueues for
poll().

> > +	list_for_each_safe(ptr, ptr2, &port->readbuf_head) {
> > +		buf = list_entry(ptr, struct virtio_serial_port_buffer, list);
> > +
> > +		/* FIXME: other buffers further in this list might
> > +		 * have data too
> > +		 */
> > +		if (count > buf->len - buf->offset)
> > +			count = buf->len - buf->offset;
> > +
> > +		ret = copy_to_user(ubuf, buf->buf + buf->offset, count);
> > +
> > +		/* Return the number of bytes actually copied */
> > +		ret = count - ret;
> > +
> > +		buf->offset += ret;
> > +
> > +		if (buf->len - buf->offset == 0) {
> > +			list_del(&buf->list);
> > +			kfree(buf->buf);
> > +			kfree(buf);
> > +		}
> > +		/* FIXME: if there's more data requested and more data
> > +		 * available, return it.
> > +		 */
> > +		break;
> 
> There's nothing wrong with short reads here.

If there were more buffers queued up, we could easily go fetch data from
them and return as much data as was requested. (BTW I've already
implemented that in my git tree.)

> > +	}
> > +	return ret;
> 
> This can return -EINTR; if you hadn't assigned ret = -EINTR unconditionally
> above, gcc would have given you a warning about that path :)

Control would come here only if there was some data to be read. We check
for list_empty above. Not enough?

> > +static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
> > +				size_t count, loff_t *offp)
> > +{
> > +	struct virtqueue *out_vq;
> > +	struct virtio_serial_port *port;
> > +	struct virtio_serial_id id;
> > +	struct vbuf *vbuf;
> > +	size_t offset, size;
> > +	ssize_t ret;
> > +	int i, id_len;
> > +
> > +	port = filp->private_data;
> > +	id.id = get_id_from_port(port);
> > +	out_vq = virtserial.out_vq;
> > +
> > +	id_len = sizeof(id);
> > +
> > +	ret = -EFBIG;
> > +	vbuf = kzalloc(sizeof(struct vbuf), GFP_KERNEL);
> > +	if (!vbuf)
> > +		return ret;
> 
> -ENOMEM is normal here.

write(2) doesn't mention a return type of ENOMEM so I avoided it. But
there's also a note saying other types of errors are possible. So I'll
switch this to -ENOMEM.

> > +static long virtserial_ioctl(struct file *filp, unsigned int ioctl,
> > +			     unsigned long arg)
> > +{
> > +	struct virtio_serial_port *port;
> > +	long ret;
> > +
> > +	port = filp->private_data;
> > +
> > +	ret = -EINVAL;
> > +	switch (ioctl) {
> > +	default:
> > +		break;
> > +	}
> > +	return ret;
> > +}
> 
> I thought -ENOTTY was normal for invalid ioctls? In which case, just don't
> implement this function.

The ioctl was introduced for getting the 'name' of the port earlier.
With that gone, there's no use for this currently. No harm removing.

And right; it should be ENOTTY.

> > +static int virtserial_release(struct inode *inode, struct file *filp)
> > +{
> > +	filp->private_data = NULL;
> > +	return 0;
> > +}
> 
> This seems redundant.

It is. I added it recently just because some other drivers were doing
it. But will remove.

> > +static unsigned int virtserial_poll(struct file *filp, poll_table *wait)
> > +{
> > +	pr_notice("%s\n", __func__);
> > +	return 0;
> > +}
> 
> And you're going to want to implement this, too.

Already in my git tree.

> > +static void virtio_serial_rx_work_handler(struct work_struct *work)
> > +{
> > +	struct virtio_serial_port *port = NULL;
> > +	struct virtio_serial_port_buffer *buf;
> > +	struct virtqueue *vq;
> > +	char *tmpbuf;
> > +	unsigned int tmplen;
> > +
> > +	vq = virtserial.in_vq;
> > +	while ((tmpbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
> > +		port = get_port_from_buf(tmpbuf);
> > +		if (!port) {
> > +			/* No valid index at start of
> > +			 * buffer. Drop it.
> > +			 */
> > +			pr_debug("%s: invalid index in buffer, %c %d\n",
> > +				 __func__, tmpbuf[0], tmpbuf[0]);
> > +			break;
> 
> leak?

Yes! and below too, in the !buf case.

> > +		}
> > +		buf = kzalloc(sizeof(struct virtio_serial_port_buffer),
> > +			      GFP_KERNEL);
> > +		if (!buf)
> > +			break;
> > +
> > +		buf->buf = tmpbuf;
> > +		buf->len = tmplen;
> > +		buf->offset = sizeof(struct virtio_serial_id);
> > +		list_add_tail(&buf->list, &port->readbuf_head);
> > +
> > +		complete(&port->have_data);
> > +	}
> > +	/* Allocate buffers for all the ones that got used up */
> > +	schedule_work(&virtserial.queue_work);
> > +}
> 
> Why do the allocation in a separate workqueue?

This is also done during probe and port hot-add case. And I guess if
some host process is sending lots of data, we could defer the allocation
to another workqueue and just allocate the empty space in one deferred
workqueue call rather than having to alloc them after each read.

> > +static __u32 get_ports_map_size(__u32 max_ports)
> > +{
> > +	return sizeof(__u32) * ((max_ports + 31) / 32);
> > +}
> 
> The __ versions are for user-visible headers only.

Ouch.

> > +static int virtserial_probe(struct virtio_device *vdev)
> > +{
> > +	struct virtqueue *vqs[3];
> 
> 3?

Remnant from control queue. Already fixed.

> > +	const char *vq_names[] = { "input", "output" };
> > +	vq_callback_t *vq_callbacks[] = { rx_intr, tx_intr };
> > +	u32 i, map;
> > +	int ret, map_i;
> > +	u32 max_nr_ports;
> > +
> > +	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
> > +					 max_nr_ports),
> > +			  &max_nr_ports,
> > +			  sizeof(max_nr_ports));
> > +	virtserial.config = kmalloc(sizeof(struct virtio_serial_config)
> > +				    + get_ports_map_size(max_nr_ports),
> > +				    GFP_KERNEL);
> 
> kmalloc not checked.

Indeed.

> > +static void virtserial_remove(struct virtio_device *vdev)
> > +{
> > +	struct list_head *ptr, *ptr2;
> > +	char *buf;
> > +	int len;
> > +
> > +	unregister_chrdev(major, "virtio-serial");
> > +	class_destroy(virtserial.class);
> > +
> > +	cancel_work_sync(&virtserial.rx_work);
> > +
> > +	/* Free up the unused buffers in the receive queue */
> > +	while ((buf = virtserial.in_vq->vq_ops->get_buf(virtserial.in_vq, &len)))
> > +		kfree(buf);
> 
> This won't quite work.  get_buf gets *used* buffers.  You need to track
> buffers yourself and delete them after del_vqs.

Oh ok. Is there no way to get them from the vq? How about a new helper
to get those buffers? /me needs to look.

> > +/* IOCTL-related */
> > +#define VIRTIO_SERIAL_IO 0xAF
> 
> ??

This is some unused IOCTL. Anyway, it should be gone.

Thanks a lot for your review, Rusty!

		Amit

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

* [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication
  2009-08-05  0:03   ` [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Rusty Russell
  2009-08-05  5:12     ` Amit Shah
@ 2009-08-05  9:58     ` Amit Shah
  1 sibling, 0 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-05  9:58 UTC (permalink / raw)
  To: Rusty Russell; +Cc: qemu-devel, kvm, virtualization

On (Wed) Aug 05 2009 [09:33:40], Rusty Russell wrote:
> On Tue, 28 Jul 2009 03:34:33 am Amit Shah wrote:
> > We expose multiple char devices ("ports") for simple communication
> > between the host userspace and guest.
> 
> Hi Amit,
> 
>    OK, seems like it's time for some serious review.  Below.

Thanks for the review, Rusty. The result is:

 drivers/char/Kconfig          |    5 -
 drivers/char/virtio_serial.c  |  160
++++++++++++++++++------------------------
 include/linux/virtio_serial.h |    3 
 3 files changed, 73 insertions(+), 95 deletions(-)

Major changes since the last version are:

- add support for poll()
- return as much data as possible for read()
- fix memleaks
- remove ioctl code
- other cleanups suggested by Rusty

Thanks,
		Amit


>From 04fa836c1d965ba57d6d662830e431e4c12637f5 Mon Sep 17 00:00:00 2001
From: Amit Shah <amit.shah@redhat.com>
Date: Wed, 5 Aug 2009 15:22:01 +0530
Subject: [PATCH] virtio_serial: A char device for simple guest <-> host communication

We expose multiple char devices ("ports") for simple communication
between the host userspace and guest.

Sample offline usages can be: poking around in a guest to find out
the file systems used, applications installed, etc. Online usages
can be sharing of clipboard data between the guest and the host,
sending information about logged-in users to the host, locking the
screen or session when a vnc session is closed, and so on.

The interface presented to guest userspace is of a simple char
device, so it can be used like this:

    fd = open("/dev/vmch0", O_RDWR);
    ret = read(fd, buf, 100);
    ret = write(fd, string, strlen(string));

Each port is to be assigned a unique function, for example, the first
4 ports may be reserved for libvirt usage, the next 4 for generic
streaming data and so on. This port-function mapping isn't finalised
yet.

For requirements, use-cases and some history see

    http://www.linux-kvm.org/page/VMchannel_Requirements

Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
 drivers/char/Kconfig          |    7 +
 drivers/char/Makefile         |    1 +
 drivers/char/virtio_serial.c  |  688 +++++++++++++++++++++++++++++++++++++++++
 include/linux/virtio_serial.h |   24 ++
 4 files changed, 720 insertions(+), 0 deletions(-)
 create mode 100644 drivers/char/virtio_serial.c
 create mode 100644 include/linux/virtio_serial.h

diff --git a/drivers/char/Kconfig b/drivers/char/Kconfig
index 6a06913..69fce0d 100644
--- a/drivers/char/Kconfig
+++ b/drivers/char/Kconfig
@@ -679,6 +679,13 @@ config VIRTIO_CONSOLE
 	help
 	  Virtio console for use with lguest and other hypervisors.
 
+config VIRTIO_SERIAL
+	tristate "Virtio serial"
+	select VIRTIO
+	help
+	  Virtio serial device driver for simple guest and host
+	  communication. Character devices at /dev/vmchNN will be
+	  created when corresponding ports are found.
 
 config HVCS
 	tristate "IBM Hypervisor Virtual Console Server support"
diff --git a/drivers/char/Makefile b/drivers/char/Makefile
index 66f779a..5e1915b 100644
--- a/drivers/char/Makefile
+++ b/drivers/char/Makefile
@@ -54,6 +54,7 @@ obj-$(CONFIG_HVC_XEN)		+= hvc_xen.o
 obj-$(CONFIG_HVC_IUCV)		+= hvc_iucv.o
 obj-$(CONFIG_HVC_UDBG)		+= hvc_udbg.o
 obj-$(CONFIG_VIRTIO_CONSOLE)	+= virtio_console.o
+obj-$(CONFIG_VIRTIO_SERIAL)	+= virtio_serial.o
 obj-$(CONFIG_RAW_DRIVER)	+= raw.o
 obj-$(CONFIG_SGI_SNSC)		+= snsc.o snsc_event.o
 obj-$(CONFIG_MSPEC)		+= mspec.o
diff --git a/drivers/char/virtio_serial.c b/drivers/char/virtio_serial.c
new file mode 100644
index 0000000..fa5ed24
--- /dev/null
+++ b/drivers/char/virtio_serial.c
@@ -0,0 +1,688 @@
+/*
+ * VirtIO Serial driver
+ *
+ * This is paravirtualised serial guest<->host communication channel
+ * for relaying short messages and events in either direction.
+ *
+ * One PCI device can have multiple serial ports so that different
+ * applications can communicate without polluting the PCI device space
+ * in the guest.
+ *
+ * Copyright (C) 2009, Red Hat, Inc.
+ *
+ * Author(s): Amit Shah <amit.shah@redhat.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License as published by
+ * the Free Software Foundation; either version 2 of the License, or
+ * (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA  02110-1301  USA
+ */
+
+#include <linux/cdev.h>
+#include <linux/completion.h>
+#include <linux/device.h>
+#include <linux/err.h>
+#include <linux/fs.h>
+#include <linux/init.h>
+#include <linux/poll.h>
+#include <linux/virtio.h>
+#include <linux/virtio_serial.h>
+#include <linux/workqueue.h>
+
+struct virtio_serial_struct {
+	struct work_struct rx_work;
+	struct work_struct tx_work;
+	struct work_struct queue_work;
+	struct work_struct config_work;
+
+	struct list_head port_head;
+	struct list_head unused_buf;
+
+	struct virtio_device *vdev;
+	struct class *class;
+	struct virtqueue *in_vq, *out_vq;
+
+	struct virtio_serial_config *config;
+};
+
+/* This struct holds individual buffers received for each port */
+struct virtio_serial_port_buffer {
+	struct list_head next;
+
+	char *buf;
+
+	size_t len; /* length of the buffer */
+	size_t offset; /* offset in the buf from which to consume data */
+};
+
+/* This struct is put in each buffer that gets passed to userspace and
+ * vice-versa
+ */
+struct virtio_serial_id {
+	u32 id; /* Port number */
+};
+
+struct virtio_serial_port {
+	/* Next port in the list */
+	struct list_head next;
+
+	/* Buffer management */
+	struct virtio_serial_port_buffer read_buf;
+	struct list_head readbuf_head;
+	wait_queue_head_t waitqueue;
+
+	/* Each port associates with a separate char device */
+	struct cdev cdev;
+	struct device *dev;
+};
+
+static struct virtio_serial_struct virtserial;
+
+static int major = 60; /* from the experimental range */
+
+static struct virtio_serial_port *get_port_from_id(u32 id)
+{
+	struct virtio_serial_port *port;
+
+	list_for_each_entry(port, &virtserial.port_head, next) {
+		if (MINOR(port->dev->devt) == id)
+			return port;
+	}
+	return NULL;
+}
+
+static int get_id_from_port(struct virtio_serial_port *port)
+{
+	return MINOR(port->dev->devt);
+}
+
+static struct virtio_serial_port *get_port_from_buf(char *buf)
+{
+	u32 id;
+
+	memcpy(&id, buf, sizeof(id));
+
+	return get_port_from_id(id);
+}
+
+
+static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
+			       size_t count, loff_t *offp)
+{
+	struct virtio_serial_port *port;
+	struct virtio_serial_port_buffer *buf, *buf2;
+	ssize_t ubuf_offset, ret;
+
+	port = filp->private_data;
+
+	ret = 0;
+	if (list_empty(&port->readbuf_head)) {
+		if (filp->f_flags & O_NONBLOCK)
+			return -EAGAIN;
+
+		ret = wait_event_interruptible(port->waitqueue,
+					       !list_empty(&port->readbuf_head));
+	}
+	if (ret < 0)
+		return ret;
+
+	ubuf_offset = 0;
+	list_for_each_entry_safe(buf, buf2, &port->readbuf_head, next) {
+		size_t copy_size;
+
+		copy_size = count;
+		if (copy_size > buf->len - buf->offset)
+			copy_size = buf->len - buf->offset;
+
+		ret = copy_to_user(ubuf + ubuf_offset, buf->buf + buf->offset,
+				   copy_size);
+
+		/* FIXME: Deal with ret != 0 */
+		/* Return the number of bytes actually copied */
+		ret = copy_size - ret;
+		buf->offset += ret;
+		ubuf_offset += ret;
+		count -= ret;
+		ret = ubuf_offset;
+
+		if (buf->len - buf->offset == 0) {
+			list_del(&buf->next);
+			kfree(buf->buf);
+			kfree(buf);
+		}
+		if (!count)
+			break;
+	}
+	return ret;
+}
+
+/* For data that exceeds PAGE_SIZE in size we should send it all in
+ * one sg to not unnecessarily split up the data. Also some (all?)
+ * vnc clients don't consume split data.
+ *
+ * If we are to keep PAGE_SIZE sized buffers, we then have to stack
+ * multiple of those in one virtio request. virtio-ring returns to us
+ * just one pointer for all the buffers. So use this struct to
+ * allocate the bufs in so that freeing this up later is easier.
+ */
+struct vbuf {
+	char **bufs;
+	struct scatterlist *sg;
+	unsigned int nent;
+};
+
+static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
+				size_t count, loff_t *offp)
+{
+	struct virtqueue *out_vq;
+	struct virtio_serial_port *port;
+	struct virtio_serial_id id;
+	struct vbuf *vbuf;
+	size_t offset, size;
+	ssize_t ret;
+	int i, id_len;
+
+	port = filp->private_data;
+	id.id = get_id_from_port(port);
+	out_vq = virtserial.out_vq;
+
+	id_len = sizeof(id);
+
+	ret = -ENOMEM;
+	vbuf = kzalloc(sizeof(struct vbuf), GFP_KERNEL);
+	if (!vbuf)
+		return ret;
+
+	/* Max. number of buffers clubbed together in one message */
+	vbuf->nent = (count + id_len + PAGE_SIZE - 1) / PAGE_SIZE;
+
+	vbuf->bufs = kzalloc(vbuf->nent, GFP_KERNEL);
+	if (!vbuf->bufs)
+		goto free_vbuf;
+
+	vbuf->sg = kzalloc(vbuf->nent, GFP_KERNEL);
+	if (!vbuf->sg)
+		goto free_bufs;
+	sg_init_table(vbuf->sg, vbuf->nent);
+
+	i = 0; /* vbuf->bufs[i] */
+	offset = 0; /* offset in the user buffer */
+	while (count - offset) {
+		size = min(count - offset + id_len, PAGE_SIZE);
+		vbuf->bufs[i] = kzalloc(size, GFP_KERNEL);
+		if (!vbuf->bufs[i]) {
+			ret = -EFBIG;
+			goto free_buffers;
+		}
+		if (id_len) {
+			memcpy(vbuf->bufs[i], &id, id_len);
+			size -= id_len;
+		}
+		ret = copy_from_user(vbuf->bufs[i] + id_len, ubuf + offset, size);
+		offset += size - ret;
+
+		sg_set_buf(&vbuf->sg[i], vbuf->bufs[i], size - ret + id_len);
+		id_len = 0; /* Pass the port id only in the first buffer */
+		i++;
+	}
+	if (out_vq->vq_ops->add_buf(out_vq, vbuf->sg, i, 0, vbuf)) {
+		/* XXX: We can't send the buffer. Report failure */
+		ret = 0;
+	}
+	/* Tell Host to go! */
+	out_vq->vq_ops->kick(out_vq);
+
+	/* We're expected to return the amount of data we wrote */
+	return offset;
+free_buffers:
+	while (i--)
+		kfree(vbuf->bufs[i]);
+	kfree(vbuf->sg);
+free_bufs:
+	kfree(vbuf->bufs);
+free_vbuf:
+	kfree(vbuf);
+	return ret;
+}
+
+static int virtserial_release(struct inode *inode, struct file *filp)
+{
+	return 0;
+}
+
+static int virtserial_open(struct inode *inode, struct file *filp)
+{
+	struct cdev *cdev = inode->i_cdev;
+	struct virtio_serial_port *port;
+
+	port = container_of(cdev, struct virtio_serial_port, cdev);
+
+	filp->private_data = port;
+	return 0;
+}
+
+static unsigned int virtserial_poll(struct file *filp, poll_table *wait)
+{
+	struct virtio_serial_port *port;
+	unsigned int ret;
+
+	port = filp->private_data;
+	poll_wait(filp, &port->waitqueue, wait);
+
+	ret = 0;
+	if (!list_empty(&port->readbuf_head))
+		ret |= POLLIN | POLLRDNORM;
+
+	return ret;
+}
+
+static const struct file_operations virtserial_fops = {
+	.owner = THIS_MODULE,
+	.open  = virtserial_open,
+	.read  = virtserial_read,
+	.write = virtserial_write,
+	.poll  = virtserial_poll,
+	.release = virtserial_release,
+};
+
+static void virtio_serial_queue_work_handler(struct work_struct *work)
+{
+	struct scatterlist sg[1];
+	struct virtqueue *vq;
+	struct virtio_serial_port_buffer *buf;
+
+	vq = virtserial.in_vq;
+	while (1) {
+		/* We have to keep track of the unused buffers
+		 * so that they can be freed when the module
+		 * is being removed
+		 */
+		buf = kzalloc(sizeof(*buf), GFP_KERNEL);
+		if (!buf)
+			break;
+		buf->buf = kzalloc(PAGE_SIZE, GFP_KERNEL);
+		if (!buf->buf) {
+			kfree(buf);
+			break;
+		}
+		sg_init_one(sg, buf->buf, PAGE_SIZE);
+
+		if (vq->vq_ops->add_buf(vq, sg, 0, 1, buf->buf) < 0) {
+			kfree(buf->buf);
+			kfree(buf);
+			break;
+		}
+		list_add_tail(&buf->next, &virtserial.unused_buf);
+	}
+	vq->vq_ops->kick(vq);
+}
+
+static void virtio_serial_rx_work_handler(struct work_struct *work)
+{
+	struct virtio_serial_port *port = NULL;
+	struct virtio_serial_port_buffer *buf;
+	struct virtqueue *vq;
+	char *tmpbuf;
+	unsigned int tmplen;
+
+	vq = virtserial.in_vq;
+	while ((tmpbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
+		list_for_each_entry(buf, &virtserial.unused_buf, next) {
+			if (tmpbuf == buf->buf)
+				break;
+		}
+		BUG_ON(!buf);
+		/* The buffer is no longer unused */
+		list_del(&buf->next);
+
+		port = get_port_from_buf(tmpbuf);
+		if (!port) {
+			/* No valid index at start of
+			 * buffer. Drop it.
+			 */
+			pr_debug("%s: invalid index in buffer, %c %d\n",
+				 __func__, tmpbuf[0], tmpbuf[0]);
+			kfree(tmpbuf);
+			kfree(buf);
+			break;
+		}
+		buf->len = tmplen;
+		buf->offset = sizeof(struct virtio_serial_id);
+		list_add_tail(&buf->next, &port->readbuf_head);
+
+		wake_up_interruptible(&port->waitqueue);
+	}
+	/* Allocate buffers for all the ones that got used up */
+	schedule_work(&virtserial.queue_work);
+}
+
+static void virtio_serial_tx_work_handler(struct work_struct *work)
+{
+	struct virtqueue *vq;
+	struct vbuf *vbuf;
+	unsigned int tmplen;
+	int i;
+
+	vq = virtserial.out_vq;
+	while ((vbuf = vq->vq_ops->get_buf(vq, &tmplen))) {
+		for (i = 0; i < vbuf->nent; i++) {
+			kfree(vbuf->bufs[i]);
+		}
+		kfree(vbuf->bufs);
+		kfree(vbuf->sg);
+		kfree(vbuf);
+	}
+}
+
+static void rx_intr(struct virtqueue *vq)
+{
+	schedule_work(&virtserial.rx_work);
+}
+
+static void tx_intr(struct virtqueue *vq)
+{
+	schedule_work(&virtserial.tx_work);
+}
+
+static void config_intr(struct virtio_device *vdev)
+{
+	schedule_work(&virtserial.config_work);
+}
+
+static u32 virtserial_get_hot_add_port(struct virtio_serial_config *config)
+{
+	u32 i;
+	u32 port_nr;
+
+	for (i = 0; i < virtserial.config->max_nr_ports / 32; i++) {
+		port_nr = ffs(config->ports_map[i] ^ virtserial.config->ports_map[i]);
+		if (port_nr)
+			break;
+	}
+	if (unlikely(!port_nr))
+		return VIRTIO_SERIAL_BAD_ID;
+
+	/* We used ffs above */
+	port_nr--;
+
+	/* FIXME: Do this only when add_port is successful */
+	virtserial.config->ports_map[i] |= 1U << port_nr;
+
+	port_nr += i * 32;
+	return port_nr;
+}
+
+static u32 virtserial_find_next_port(u32 *map, int *map_i)
+{
+	u32 port_nr;
+
+	while (1) {
+		port_nr = ffs(*map);
+		if (port_nr)
+			break;
+
+		if (unlikely(*map_i >= virtserial.config->max_nr_ports / 32))
+			return VIRTIO_SERIAL_BAD_ID;
+		++*map_i;
+		*map = virtserial.config->ports_map[*map_i];
+	}
+	/* We used ffs above */
+	port_nr--;
+
+	/* FIXME: Do this only when add_port is successful / reset bit
+	 * in config space if add_port was unsuccessful
+	 */
+	*map &= ~(1U << port_nr);
+
+	port_nr += *map_i * 32;
+	return port_nr;
+}
+
+static int virtserial_add_port(u32 port_nr)
+{
+	struct virtio_serial_port *port;
+	dev_t devt;
+	int ret;
+
+	port = kzalloc(sizeof(struct virtio_serial_port), GFP_KERNEL);
+	if (!port)
+		return -ENOMEM;
+
+	devt = MKDEV(major, port_nr);
+	cdev_init(&port->cdev, &virtserial_fops);
+
+	ret = register_chrdev_region(devt, 1, "virtio-serial");
+	if (ret < 0) {
+		pr_err("%s: error registering chrdev region, ret = %d\n",
+		       __func__, ret);
+		goto free_cdev;
+	}
+	ret = cdev_add(&port->cdev, devt, 1);
+	if (ret < 0) {
+		pr_err("%s: error adding cdev, ret = %d\n", __func__, ret);
+		goto free_cdev;
+	}
+	port->dev = device_create(virtserial.class, NULL, devt, NULL,
+				  "vmch%u", port_nr);
+	if (IS_ERR(port->dev)) {
+		ret = PTR_ERR(port->dev);
+		pr_err("%s: Error creating device, ret = %d\n", __func__, ret);
+		goto free_cdev;
+	}
+	INIT_LIST_HEAD(&port->readbuf_head);
+	init_waitqueue_head(&port->waitqueue);
+
+	list_add_tail(&port->next, &virtserial.port_head);
+
+	pr_info("virtio-serial port found at id %u\n", port_nr);
+
+	return 0;
+free_cdev:
+	unregister_chrdev(major, "virtio-serial");
+	return ret;
+}
+
+static u32 get_ports_map_size(u32 max_ports)
+{
+	return sizeof(u32) * ((max_ports + 31) / 32);
+}
+
+static void virtio_serial_config_work_handler(struct work_struct *work)
+{
+	struct virtio_serial_config *virtserconf;
+	struct virtio_device *vdev = virtserial.vdev;
+	u32 i, port_nr;
+	int ret;
+
+	virtserconf = kmalloc(sizeof(struct virtio_serial_config) +
+			      get_ports_map_size(virtserial.config->max_nr_ports),
+			      GFP_KERNEL);
+	vdev->config->get(vdev,
+			  offsetof(struct virtio_serial_config, nr_active_ports),
+			  &virtserconf->nr_active_ports,
+			  sizeof(virtserconf->nr_active_ports));
+	vdev->config->get(vdev,
+			  offsetof(struct virtio_serial_config, ports_map),
+			  virtserconf->ports_map,
+			  get_ports_map_size(virtserial.config->max_nr_ports));
+
+	/* Hot-add ports */
+	for (i = virtserial.config->nr_active_ports;
+	     i < virtserconf->nr_active_ports; i++) {
+		port_nr = virtserial_get_hot_add_port(virtserconf);
+		if (port_nr == VIRTIO_SERIAL_BAD_ID)
+			continue;
+		ret = virtserial_add_port(port_nr);
+		if (!ret)
+			virtserial.config->nr_active_ports++;
+	}
+	kfree(virtserconf);
+}
+
+static int virtserial_probe(struct virtio_device *vdev)
+{
+	struct virtqueue *vqs[2];
+	const char *vq_names[] = { "input", "output" };
+	vq_callback_t *vq_callbacks[] = { rx_intr, tx_intr };
+	u32 i, map;
+	int ret, map_i;
+	u32 max_nr_ports;
+
+	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
+					 max_nr_ports),
+			  &max_nr_ports,
+			  sizeof(max_nr_ports));
+	virtserial.config = kmalloc(sizeof(struct virtio_serial_config)
+				    + get_ports_map_size(max_nr_ports),
+				    GFP_KERNEL);
+	if (!virtserial.config)
+		return -ENOMEM;
+	virtserial.config->max_nr_ports = max_nr_ports;
+
+	vdev->config->get(vdev, offsetof(struct virtio_serial_config,
+					 nr_active_ports),
+			  &virtserial.config->nr_active_ports,
+			  sizeof(virtserial.config->nr_active_ports));
+	vdev->config->get(vdev,
+			  offsetof(struct virtio_serial_config, ports_map),
+			  virtserial.config->ports_map,
+			  get_ports_map_size(max_nr_ports));
+
+	virtserial.vdev = vdev;
+
+	ret = vdev->config->find_vqs(vdev, 2, vqs, vq_callbacks, vq_names);
+	if (ret)
+		goto fail;
+
+	virtserial.in_vq = vqs[0];
+	virtserial.out_vq = vqs[1];
+
+	INIT_LIST_HEAD(&virtserial.port_head);
+	INIT_LIST_HEAD(&virtserial.unused_buf);
+
+	map_i = 0;
+	map = virtserial.config->ports_map[map_i];
+	for (i = 0; i < virtserial.config->nr_active_ports; i++) {
+		u32 port_nr;
+
+		port_nr = virtserial_find_next_port(&map, &map_i);
+		if (unlikely(port_nr == VIRTIO_SERIAL_BAD_ID))
+			continue;
+
+		virtserial_add_port(port_nr);
+	}
+	INIT_WORK(&virtserial.rx_work, &virtio_serial_rx_work_handler);
+	INIT_WORK(&virtserial.tx_work, &virtio_serial_tx_work_handler);
+	INIT_WORK(&virtserial.queue_work, &virtio_serial_queue_work_handler);
+	INIT_WORK(&virtserial.config_work, &virtio_serial_config_work_handler);
+
+	/* Allocate pages to fill the receive queue */
+	schedule_work(&virtserial.queue_work);
+
+	return 0;
+fail:
+	return ret;
+}
+
+
+static void virtserial_remove_port_data(struct virtio_serial_port *port)
+{
+	struct virtio_serial_port_buffer *buf, *buf2;
+
+	device_destroy(virtserial.class, port->dev->devt);
+	unregister_chrdev_region(port->dev->devt, 1);
+	cdev_del(&port->cdev);
+
+	/* Remove the buffers in which we have unconsumed data */
+	list_for_each_entry_safe(buf, buf2, &port->readbuf_head, next) {
+		list_del(&buf->next);
+		kfree(buf->buf);
+		kfree(buf);
+	}
+}
+
+static void virtserial_remove(struct virtio_device *vdev)
+{
+	struct virtio_serial_port *port, *port2;
+	struct virtio_serial_port_buffer *buf, *buf2;
+	char *buff;
+	int len;
+
+	unregister_chrdev(major, "virtio-serial");
+	class_destroy(virtserial.class);
+
+	cancel_work_sync(&virtserial.rx_work);
+
+	/* Free up the buffers in the 'received' queue */
+	while ((buff = virtserial.in_vq->vq_ops->get_buf(virtserial.in_vq, &len)))
+		kfree(buff);
+
+	vdev->config->del_vqs(vdev);
+
+	/* Free up the buffers that were unused */
+	list_for_each_entry_safe(buf, buf2, &virtserial.unused_buf, next) {
+		list_del(&buf->next);
+		kfree(buf->buf);
+		kfree(buf);
+	}
+	list_for_each_entry_safe(port, port2, &virtserial.port_head, next) {
+		list_del(&port->next);
+		virtserial_remove_port_data(port);
+		kfree(port);
+	}
+	kfree(virtserial.config);
+}
+
+static struct virtio_device_id id_table[] = {
+	{ VIRTIO_ID_SERIAL, VIRTIO_DEV_ANY_ID },
+	{ 0 },
+};
+
+static struct virtio_driver virtio_serial = {
+  //	.feature_table = virtserial_features,
+  //	.feature_table_size = ARRAY_SIZE(virtserial_features),
+	.driver.name =	KBUILD_MODNAME,
+	.driver.owner =	THIS_MODULE,
+	.id_table =	id_table,
+	.probe =	virtserial_probe,
+	.remove =	virtserial_remove,
+	.config_changed = config_intr,
+};
+
+static int __init init(void)
+{
+	int ret;
+
+	virtserial.class = class_create(THIS_MODULE, "virtio-serial");
+	if (IS_ERR(virtserial.class)) {
+		pr_err("Error creating virtio-serial class\n");
+		ret = PTR_ERR(virtserial.class);
+		return ret;
+	}
+	ret = register_virtio_driver(&virtio_serial);
+	if (ret) {
+		class_destroy(virtserial.class);
+		return ret;
+	}
+	return 0;
+}
+
+static void __exit fini(void)
+{
+	unregister_virtio_driver(&virtio_serial);
+}
+module_init(init);
+module_exit(fini);
+
+MODULE_DEVICE_TABLE(virtio, id_table);
+MODULE_DESCRIPTION("Virtio serial driver");
+MODULE_LICENSE("GPL");
diff --git a/include/linux/virtio_serial.h b/include/linux/virtio_serial.h
new file mode 100644
index 0000000..89b639b
--- /dev/null
+++ b/include/linux/virtio_serial.h
@@ -0,0 +1,24 @@
+#ifndef _LINUX_VIRTIO_SERIAL_H
+#define _LINUX_VIRTIO_SERIAL_H
+#include <linux/types.h>
+#include <linux/virtio_config.h>
+
+/* Guest kernel - Host interface */
+
+/* The ID for virtio serial */
+#define VIRTIO_ID_SERIAL		7
+
+#define VIRTIO_SERIAL_BAD_ID		(~(u32)0)
+
+struct virtio_serial_config {
+	__u32 max_nr_ports;
+	__u32 nr_active_ports;
+	__u32 ports_map[0 /* (max_nr_ports + 31) / 32 */];
+};
+
+#ifdef __KERNEL__
+
+/* Guest kernel - Guest userspace interface */
+
+#endif /* __KERNEL__ */
+#endif /* _LINUX_VIRTIO_SERIAL_H */
-- 
1.6.2.5

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-03 19:57           ` Anthony Liguori
@ 2009-08-05 17:57             ` Jamie Lokier
  2009-08-05 18:00               ` Anthony Liguori
  2009-08-06 10:35               ` Amit Shah
  2009-08-05 18:32             ` Richard W.M. Jones
  1 sibling, 2 replies; 62+ messages in thread
From: Jamie Lokier @ 2009-08-05 17:57 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: Amit Shah, virtualization, Richard W.M. Jones, kvm, qemu-devel

Anthony Liguori wrote:
> Richard W.M. Jones wrote:
> Have you considered using a usb serial device?  Something attractive 
> about it is that a productid/vendorid can be specified which means that 
> you can use that as a method of enumerating devices.
> 
> Hot add/remove is supported automagically.

The same applies to PCI: productid/vendorid (and subids);
PCI hotplug is possible though not as native as USB.

Here's another idea: Many devices these days have a serial number or
id string.  E.g. USB storage, ATA drives, media cards, etc.  Linux
these days creates alias device nodes which include the id string in
the device name.  E.g. /dev/disks/by-id/ata-FUJITSU_MHV2100BH_NWAQT662615H

So in addition to (or instead of) /dev/vmch0, /dev/vmch1 etc.,
Linux guests could easily generate:

    /dev/vmchannel/by-role/clipboard-0
    /dev/vmchannel/by-role/gueststats-0
    /dev/vmchannel/by-role/vmmanager-0

It's not necessary to do this at the beginning.  All that is needed is
to provide enough id information that will appear in /sys/..., so that
that a udev policy for naming devices can be created at some later date.

-- Jamie

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-07-29  7:48               ` Gleb Natapov
@ 2009-08-05 18:00                 ` Jamie Lokier
  0 siblings, 0 replies; 62+ messages in thread
From: Jamie Lokier @ 2009-08-05 18:00 UTC (permalink / raw)
  To: Gleb Natapov; +Cc: kvm, qemu-devel, virtualization, Amit Shah

Gleb Natapov wrote:
> On Wed, Jul 29, 2009 at 01:14:18PM +0530, Amit Shah wrote:
> > But why do we want to limit the device to only one port? It's not too
> > complex supporting additional ones.
> > 
> > As I see it qemu and the kernel should provide the basic abstraction for
> > the userspace to go do its job. Why create unnecessary barriers?
> > 
> I agree. If userspace wants it may use only one channel and demultiplex
> messages by itself, but we shouldn't force it to. Also one of the
> requirements for virtio-serial is to have connect disconnect
> notifications. It is not possible with demultiplexing in the userspace.

I agree too, for all those reasons.

However it would be useful if the devices provided a simpler way to be
found by guest applications than /dev/vmch0, vmch1, vmch2...

On Linux udev provides a sane way to find devices according to roles,
subtypes, serial numbers, whatever you want, if the appropriate id
codes are available from the devices and put into /sys/* by the kernel
driver.  That would make the devices much more useful to independent
applications, imho.

-- Jamie

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-05 17:57             ` Jamie Lokier
@ 2009-08-05 18:00               ` Anthony Liguori
  2009-08-06 10:38                 ` Amit Shah
  2009-08-06 10:35               ` Amit Shah
  1 sibling, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-05 18:00 UTC (permalink / raw)
  To: Jamie Lokier
  Cc: Amit Shah, virtualization, Richard W.M. Jones, kvm, qemu-devel

Jamie Lokier wrote:
> Anthony Liguori wrote:
>   
>> Richard W.M. Jones wrote:
>> Have you considered using a usb serial device?  Something attractive 
>> about it is that a productid/vendorid can be specified which means that 
>> you can use that as a method of enumerating devices.
>>
>> Hot add/remove is supported automagically.
>>     
>
> The same applies to PCI: productid/vendorid (and subids);
> PCI hotplug is possible though not as native as USB.
>   

What's nice about USB is that HID specifies quite a few functional 
generic devices that can be extended to increase functionality.  This 
means you can implement a more sophisticated usb device that satisfies 
the serial interface, provide a special more featureful driver for 
Linux, and just use normal serial for Windows.

The downside is that USB emulation stinks.

> Here's another idea: Many devices these days have a serial number or
> id string.  E.g. USB storage, ATA drives, media cards, etc.  Linux
> these days creates alias device nodes which include the id string in
> the device name.  E.g. /dev/disks/by-id/ata-FUJITSU_MHV2100BH_NWAQT662615H
>
> So in addition to (or instead of) /dev/vmch0, /dev/vmch1 etc.,
> Linux guests could easily generate:
>
>     /dev/vmchannel/by-role/clipboard-0
>     /dev/vmchannel/by-role/gueststats-0
>     /dev/vmchannel/by-role/vmmanager-0
>
> It's not necessary to do this at the beginning.  All that is needed is
> to provide enough id information that will appear in /sys/..., so that
> that a udev policy for naming devices can be created at some later date.
>   

Well my thinking is that the "clipboard" device actually becomes a USB 
serial device.  It's easy to enumerate and detect via the existing Linux 
infrastructure.  Plus usb drivers can be implemented in userspace which 
is a nice plus (cross platform too via libusb).

Regards,

Anthony Liguori

> -- Jamie
>   

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-03 19:57           ` Anthony Liguori
  2009-08-05 17:57             ` Jamie Lokier
@ 2009-08-05 18:32             ` Richard W.M. Jones
  1 sibling, 0 replies; 62+ messages in thread
From: Richard W.M. Jones @ 2009-08-05 18:32 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: qemu-devel, kvm, virtualization

On Mon, Aug 03, 2009 at 02:57:01PM -0500, Anthony Liguori wrote:
> Richard W.M. Jones wrote:
>> On Mon, Jul 27, 2009 at 06:44:28PM -0500, Anthony Liguori wrote:
>>   
>>> It really suggests that you need _one_ vmchannel that's exposed to   
>>> userspace with a single userspace daemon that consumes it.
>>>     
>>
>> ... or a more flexible API.  I don't like having fixed /dev/vmch*
>> devices either.
>>   
>
> Have you considered using a usb serial device?  Something attractive  
> about it is that a productid/vendorid can be specified which means that  
> you can use that as a method of enumerating devices.
>
> Hot add/remove is supported automagically.

[ Meant to reply to this two days ago :-( ]

We're using -net channel ^W guestfwd in libguestfs now.  Apart from
the problem with using the "new syntax", which I hope to get around
to resolving some day, it performs quite well.

The userspace API is somewhat annoying.

Hot add/remove isn't a concern for us right now, nor is migration.
Since we can throw up new qemu-based appliances in a few seconds.

Rich.

-- 
Richard Jones, Emerging Technologies, Red Hat  http://et.redhat.com/~rjones
New in Fedora 11: Fedora Windows cross-compiler. Compile Windows
programs, test, and build Windows installers. Over 70 libraries supprt'd
http://fedoraproject.org/wiki/MinGW http://www.annexia.org/fedora_mingw

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-05 17:57             ` Jamie Lokier
  2009-08-05 18:00               ` Anthony Liguori
@ 2009-08-06 10:35               ` Amit Shah
  1 sibling, 0 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-06 10:35 UTC (permalink / raw)
  To: Jamie Lokier; +Cc: kvm, virtualization, Richard W.M. Jones, qemu-devel

On (Wed) Aug 05 2009 [18:57:13], Jamie Lokier wrote:
> Anthony Liguori wrote:
> > Richard W.M. Jones wrote:
> > Have you considered using a usb serial device?  Something attractive 
> > about it is that a productid/vendorid can be specified which means that 
> > you can use that as a method of enumerating devices.
> > 
> > Hot add/remove is supported automagically.
> 
> The same applies to PCI: productid/vendorid (and subids);
> PCI hotplug is possible though not as native as USB.
> 
> Here's another idea: Many devices these days have a serial number or
> id string.  E.g. USB storage, ATA drives, media cards, etc.  Linux
> these days creates alias device nodes which include the id string in
> the device name.  E.g. /dev/disks/by-id/ata-FUJITSU_MHV2100BH_NWAQT662615H
> 
> So in addition to (or instead of) /dev/vmch0, /dev/vmch1 etc.,
> Linux guests could easily generate:
> 
>     /dev/vmchannel/by-role/clipboard-0
>     /dev/vmchannel/by-role/gueststats-0
>     /dev/vmchannel/by-role/vmmanager-0

That's interesting; worth a thought. When we actually have all the
parties together (libvirt, libguestfs, qemu) to decide which ports need
to act as which transports, we'll be able to add this.

> It's not necessary to do this at the beginning.  All that is needed is
> to provide enough id information that will appear in /sys/..., so that
> that a udev policy for naming devices can be created at some later date.

True.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-05 18:00               ` Anthony Liguori
@ 2009-08-06 10:38                 ` Amit Shah
  2009-08-06 13:29                   ` Anthony Liguori
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-06 10:38 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: qemu-devel, Richard W.M. Jones, kvm, virtualization

On (Wed) Aug 05 2009 [13:00:57], Anthony Liguori wrote:
> Jamie Lokier wrote:
>> Anthony Liguori wrote:
>>   
>>> Richard W.M. Jones wrote:
>>> Have you considered using a usb serial device?  Something attractive  
>>> about it is that a productid/vendorid can be specified which means 
>>> that you can use that as a method of enumerating devices.
>>>
>>> Hot add/remove is supported automagically.
>>>     
>>
>> The same applies to PCI: productid/vendorid (and subids);
>> PCI hotplug is possible though not as native as USB.
>>   
>
> What's nice about USB is that HID specifies quite a few functional  
> generic devices that can be extended to increase functionality.  This  
> means you can implement a more sophisticated usb device that satisfies  
> the serial interface, provide a special more featureful driver for  
> Linux, and just use normal serial for Windows.
>
> The downside is that USB emulation stinks.

And the virtio code is pretty simple and self-contained. I don't see why
we'd restrict us more to use something else.

>> Here's another idea: Many devices these days have a serial number or
>> id string.  E.g. USB storage, ATA drives, media cards, etc.  Linux
>> these days creates alias device nodes which include the id string in
>> the device name.  E.g. /dev/disks/by-id/ata-FUJITSU_MHV2100BH_NWAQT662615H
>>
>> So in addition to (or instead of) /dev/vmch0, /dev/vmch1 etc.,
>> Linux guests could easily generate:
>>
>>     /dev/vmchannel/by-role/clipboard-0
>>     /dev/vmchannel/by-role/gueststats-0
>>     /dev/vmchannel/by-role/vmmanager-0
>>
>> It's not necessary to do this at the beginning.  All that is needed is
>> to provide enough id information that will appear in /sys/..., so that
>> that a udev policy for naming devices can be created at some later date.
>
> Well my thinking is that the "clipboard" device actually becomes a USB  
> serial device.  It's easy to enumerate and detect via the existing Linux  
> infrastructure.  Plus usb drivers can be implemented in userspace which  
> is a nice plus (cross platform too via libusb).

Sure; but there's been no resistance from anyone from including the
virtio-serial device driver so maybe we don't need to discuss that.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-06 10:38                 ` Amit Shah
@ 2009-08-06 13:29                   ` Anthony Liguori
  2009-08-06 13:41                     ` Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-06 13:29 UTC (permalink / raw)
  To: Amit Shah
  Cc: kvm, Rusty Russell, qemu-devel, Richard W.M. Jones,
	virtualization

Amit Shah wrote:
> Sure; but there's been no resistance from anyone from including the
> virtio-serial device driver so maybe we don't need to discuss that.
>   

There certainly is from me.  The userspace interface is not reasonable 
for guest applications to use.

Regards,

Anthony Liguori

> 		Amit
>   

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-06 13:29                   ` Anthony Liguori
@ 2009-08-06 13:41                     ` Amit Shah
  2009-08-06 13:58                       ` Anthony Liguori
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-06 13:41 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: kvm, Rusty Russell, qemu-devel, Richard W.M. Jones,
	virtualization

On (Thu) Aug 06 2009 [08:29:40], Anthony Liguori wrote:
> Amit Shah wrote:
>> Sure; but there's been no resistance from anyone from including the
>> virtio-serial device driver so maybe we don't need to discuss that.
>>   
>
> There certainly is from me.  The userspace interface is not reasonable  
> for guest applications to use.

One example that would readily come to mind is dbus. A daemon running on
the guest that reads data off the port and interacts with the desktop by
appropriate dbus commands. All that's needed is a stream of bytes and
virtio-serial provides just that.

Any more complexity could easily be handled in userspace.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-06 13:41                     ` Amit Shah
@ 2009-08-06 13:58                       ` Anthony Liguori
  2009-08-06 14:04                         ` Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-06 13:58 UTC (permalink / raw)
  To: Amit Shah
  Cc: kvm, Rusty Russell, qemu-devel, Richard W.M. Jones,
	virtualization

Amit Shah wrote:
> On (Thu) Aug 06 2009 [08:29:40], Anthony Liguori wrote:
>   
>> Amit Shah wrote:
>>     
>>> Sure; but there's been no resistance from anyone from including the
>>> virtio-serial device driver so maybe we don't need to discuss that.
>>>   
>>>       
>> There certainly is from me.  The userspace interface is not reasonable  
>> for guest applications to use.
>>     
>
> One example that would readily come to mind is dbus. A daemon running on
> the guest that reads data off the port and interacts with the desktop by
> appropriate dbus commands. All that's needed is a stream of bytes and
> virtio-serial provides just that.
>   

dbus runs as an unprivileged user, how does dbus know which 
virtio-serial port to open and who sets the permissions on that port?

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-06 13:58                       ` Anthony Liguori
@ 2009-08-06 14:04                         ` Amit Shah
  2009-08-06 17:37                           ` Jamie Lokier
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-06 14:04 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: Rusty Russell, virtualization, qemu-devel, kvm,
	Richard W.M. Jones

On (Thu) Aug 06 2009 [08:58:01], Anthony Liguori wrote:
> Amit Shah wrote:
>> On (Thu) Aug 06 2009 [08:29:40], Anthony Liguori wrote:
>>   
>>> Amit Shah wrote:
>>>     
>>>> Sure; but there's been no resistance from anyone from including the
>>>> virtio-serial device driver so maybe we don't need to discuss that.
>>>>         
>>> There certainly is from me.  The userspace interface is not 
>>> reasonable  for guest applications to use.
>>>     
>>
>> One example that would readily come to mind is dbus. A daemon running on
>> the guest that reads data off the port and interacts with the desktop by
>> appropriate dbus commands. All that's needed is a stream of bytes and
>> virtio-serial provides just that.
>>   
>
> dbus runs as an unprivileged user, how does dbus know which  
> virtio-serial port to open and who sets the permissions on that port?

The permission part can be handled by package maintainers and sysadmins
via udev policies.

So all data destined for dbus consumption gets to a daemon and that
daemon then sends it over to dbus.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-06 14:04                         ` Amit Shah
@ 2009-08-06 17:37                           ` Jamie Lokier
  2009-08-07  6:38                             ` Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Jamie Lokier @ 2009-08-06 17:37 UTC (permalink / raw)
  To: Amit Shah
  Cc: kvm, Rusty Russell, Richard W.M. Jones, qemu-devel,
	virtualization

Amit Shah wrote:
> On (Thu) Aug 06 2009 [08:58:01], Anthony Liguori wrote:
> > Amit Shah wrote:
> >> On (Thu) Aug 06 2009 [08:29:40], Anthony Liguori wrote:
> >>   
> >>> Amit Shah wrote:
> >>>     
> >>>> Sure; but there's been no resistance from anyone from including the
> >>>> virtio-serial device driver so maybe we don't need to discuss that.
> >>>>         
> >>> There certainly is from me.  The userspace interface is not 
> >>> reasonable  for guest applications to use.
> >>>     
> >>
> >> One example that would readily come to mind is dbus. A daemon running on
> >> the guest that reads data off the port and interacts with the desktop by
> >> appropriate dbus commands. All that's needed is a stream of bytes and
> >> virtio-serial provides just that.
> >>   
> >
> > dbus runs as an unprivileged user, how does dbus know which  
> > virtio-serial port to open and who sets the permissions on that port?
> 
> The permission part can be handled by package maintainers and sysadmins
> via udev policies.
> 
> So all data destined for dbus consumption gets to a daemon and that
> daemon then sends it over to dbus.

virtio-serial is nice, easy, simple and versatile.  We like that; it
should stay that way.

dbus isn't a good match for this.

dbus is not intended for communication between hosts, by design.

It depends on per-app configuration files in
/etc/dbus/{session,system}.d/, which are expected to match the
installed services.

For this, the guest's files in /etc/dbus/ would have to match the QEMU
host host services in detail.  dbus doesn't have a good mechanism for
copying with version skew between both of them, because normally
everything resides on the same machine and the config and service are
updated at the same time.  This is hard to guarantee with a VM.

Apart from dbus, hard-coded meanings of small N in /dev/vmchN are
asking for trouble.  It is bound to break when widely deployed and
guest/host configs don't match.  It also doesn't fit comfortably when
you have, say, bob and alice both logged in with desktops on separate
VTs.  Clashes are inevitable, as third-party apps pick N values for
themselves then get distributed - unless N values can be large
(/dev/vmch44324 == kernelstats...).

Sysadmins shouldn't have to hand-configure each app, and shouldn't
have to repair clashes in defaults.  Just Work is better.

virtio-serial is nice.  The only ugly part is _finding_ the right
/dev/vmchN.

Fortunately, _any_ out-of-band id string or id number makes it perfect.

An option to specify PCI vendor/product ids in the QEMU host
configuration is good enough.

An option to specify one or more id strings is nicer.

Finally, Anthony hit on an interesting idea with USB.  Emulating USB
sucks.  But USB's _descriptors_ are quite effective, and the USB basic
protocol is quite reasonable too.

Descriptors are just a binary blob in a particular format, which
describe a device and also say what it supports, and what standard
interfaces can be used with it too.  Bluetooth is similar; they might
even use the same byte format, I'm not sure.

All the code for parsing USB descriptors is already present in guest
kernels, and the code for making appropriate device nodes and
launching apps is already in udev.  libusb also allows devices to be
used without a kernel driver, and is cross-platform.  There are plenty
of examples of creating USB descriptors in QEMU, and may be the code
can be reused.

The only down side of USB is that emulating it sucks :-)  That's mainly
due to the host controllers, and the way interrupts use polling.

So here's a couple of ideas:

   - virtio-usb, using virtio instead of a hardware USB host
     controller.  That would provide all the features of USB
     naturally, like hotplug, device binding, access from userspace,
     but with high performance, low overhead, and no interrupt polling.

     You'd even have the option of cross-platform guest apps, as well
     as working on all Linux versions, by emulating a host controller
     when the guest doesn't have virtio-usb.

     As a bonus, existing USB support would be accelerated.

   - virtio-serial providing a binary id blob, whose format is the
     same as USB descriptors.  Reuse the guest's USB parsing and
     binding to find and identify, but the actual device functionality
     would just be a byte pipe.

     That might be simple, as all it involves is a blob passed to the
     guest from QEMU.  QEMU would build the id blob, maybe reusing
     existing USB code, and the guest would parse the blob as it
     already does for USB devices, with udev creating devices as it
     already does.

-- Jamie

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-06 17:37                           ` Jamie Lokier
@ 2009-08-07  6:38                             ` Amit Shah
  2009-08-07 14:14                               ` Anthony Liguori
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-07  6:38 UTC (permalink / raw)
  To: Jamie Lokier
  Cc: Rusty Russell, virtualization, Richard W.M. Jones, kvm,
	qemu-devel

On (Thu) Aug 06 2009 [18:37:40], Jamie Lokier wrote:
> Amit Shah wrote:
> > On (Thu) Aug 06 2009 [08:58:01], Anthony Liguori wrote:
> > > Amit Shah wrote:
> > >> On (Thu) Aug 06 2009 [08:29:40], Anthony Liguori wrote:
> > >>   
> > >>> Amit Shah wrote:
> > >>>     
> > >>>> Sure; but there's been no resistance from anyone from including the
> > >>>> virtio-serial device driver so maybe we don't need to discuss that.
> > >>>>         
> > >>> There certainly is from me.  The userspace interface is not 
> > >>> reasonable  for guest applications to use.
> > >>>     
> > >>
> > >> One example that would readily come to mind is dbus. A daemon running on
> > >> the guest that reads data off the port and interacts with the desktop by
> > >> appropriate dbus commands. All that's needed is a stream of bytes and
> > >> virtio-serial provides just that.
> > >>   
> > >
> > > dbus runs as an unprivileged user, how does dbus know which  
> > > virtio-serial port to open and who sets the permissions on that port?
> > 
> > The permission part can be handled by package maintainers and sysadmins
> > via udev policies.
> > 
> > So all data destined for dbus consumption gets to a daemon and that
> > daemon then sends it over to dbus.
> 
> virtio-serial is nice, easy, simple and versatile.  We like that; it
> should stay that way.
> 
> dbus isn't a good match for this.
> 
> dbus is not intended for communication between hosts, by design.

Oh; I don't mean to say dbus on the host will communicate directly with
dbus on the guest via virtio-serial. I'm just saying there'll be some
daemon on the guest and if there's a request for, say, updating guest
clipboard with some contents, it can be passed on to dbus on the guest.

> It depends on per-app configuration files in
> /etc/dbus/{session,system}.d/, which are expected to match the
> installed services.
> 
> For this, the guest's files in /etc/dbus/ would have to match the QEMU
> host host services in detail.  dbus doesn't have a good mechanism for
> copying with version skew between both of them, because normally
> everything resides on the same machine and the config and service are
> updated at the same time.  This is hard to guarantee with a VM.

Right. Not proposing this at all.

> Apart from dbus, hard-coded meanings of small N in /dev/vmchN are
> asking for trouble.  It is bound to break when widely deployed and

It's no different from the way major-minor numbering works on the Linux
kernel: they uniquely identify a device. Or the way tcp port number
works. In the same way, /dev/vmchN will uniquely identify some function
over a port. You idea of introducing a symlink to /dev/clipboard is a
good one and it takes one udev rule to generate that link.

> guest/host configs don't match.  It also doesn't fit comfortably when
> you have, say, bob and alice both logged in with desktops on separate
> VTs.  Clashes are inevitable, as third-party apps pick N values for
> themselves then get distributed - unless N values can be large
> (/dev/vmch44324 == kernelstats...).

Hm, so there can be one daemon on the guest handling all clipboard
events. There's some work done already by the fast-user-switch support
and that can be extended to daemons that talk over virtio-serial.

> Sysadmins shouldn't have to hand-configure each app, and shouldn't
> have to repair clashes in defaults.  Just Work is better.

No; they shouldn't need to.

> virtio-serial is nice.  The only ugly part is _finding_ the right
> /dev/vmchN.
> 
> Fortunately, _any_ out-of-band id string or id number makes it perfect.
> 
> An option to specify PCI vendor/product ids in the QEMU host
> configuration is good enough.
> 
> An option to specify one or more id strings is nicer.
> 
> Finally, Anthony hit on an interesting idea with USB.  Emulating USB
> sucks.  But USB's _descriptors_ are quite effective, and the USB basic
> protocol is quite reasonable too.
> 
> Descriptors are just a binary blob in a particular format, which
> describe a device and also say what it supports, and what standard
> interfaces can be used with it too.  Bluetooth is similar; they might
> even use the same byte format, I'm not sure.

And doing something similar is akin to populating some files in /sys.

> All the code for parsing USB descriptors is already present in guest
> kernels, and the code for making appropriate device nodes and
> launching apps is already in udev.  libusb also allows devices to be
> used without a kernel driver, and is cross-platform.  There are plenty
> of examples of creating USB descriptors in QEMU, and may be the code
> can be reused.
> 
> The only down side of USB is that emulating it sucks :-)  That's mainly
> due to the host controllers, and the way interrupts use polling.
> 
> So here's a couple of ideas:
> 
>    - virtio-usb, using virtio instead of a hardware USB host
>      controller.  That would provide all the features of USB
>      naturally, like hotplug, device binding, access from userspace,
>      but with high performance, low overhead, and no interrupt polling.

I wonder how that's any different or less complex that virtio-serial.
Essentially the idea is the same. It's just the name that's different,
the way I see it.

>      You'd even have the option of cross-platform guest apps, as well
>      as working on all Linux versions, by emulating a host controller
>      when the guest doesn't have virtio-usb.
> 
>      As a bonus, existing USB support would be accelerated.
> 
>    - virtio-serial providing a binary id blob, whose format is the
>      same as USB descriptors.  Reuse the guest's USB parsing and
>      binding to find and identify, but the actual device functionality
>      would just be a byte pipe.
> 
>      That might be simple, as all it involves is a blob passed to the
>      guest from QEMU.  QEMU would build the id blob, maybe reusing
>      existing USB code, and the guest would parse the blob as it
>      already does for USB devices, with udev creating devices as it
>      already does.

Hm, making a very simple transport into something complicated involving
a few more subsystems just to expose a usb device which functions as a
char device in the end. I don't see a big benefit. People are deploying
virtio anyway for block and net. I don't see why using the same
transport for a char device would be an impediment.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-07  6:38                             ` Amit Shah
@ 2009-08-07 14:14                               ` Anthony Liguori
  2009-08-10  6:55                                 ` Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-07 14:14 UTC (permalink / raw)
  To: Amit Shah
  Cc: kvm, qemu-devel, Rusty Russell, Richard W.M. Jones,
	virtualization

Amit Shah wrote:
> On (Thu) Aug 06 2009 [18:37:40], Jamie Lokier wrote:
>   
>> Apart from dbus, hard-coded meanings of small N in /dev/vmchN are
>> asking for trouble.  It is bound to break when widely deployed and
>>     
>
> It's no different from the way major-minor numbering works on the Linux
> kernel: they uniquely identify a device.

Bad example.  Quite a lot of modern devices drivers are using dynamic 
major/minor numbers because they have proven to be such a pain in the 
butt.  That's why we have more sophisticated mechanisms like udev for 
userspace to make use of.

We'll definitely need some way to support dynamic vmchannels.  Static 
allocation of ports is just not going to work.  If we did a userspace 
daemon, I'd suggest using some sort of universal identifier that's easy 
to manage in a distributed fashion.  Like a reverse fqdn.

So for instance, I could have an "com.ibm.my-awesome-channel" and never 
have to worry about conflicts.

>> guest/host configs don't match.  It also doesn't fit comfortably when
>> you have, say, bob and alice both logged in with desktops on separate
>> VTs.  Clashes are inevitable, as third-party apps pick N values for
>> themselves then get distributed - unless N values can be large
>> (/dev/vmch44324 == kernelstats...).
>>     
>
> Hm, so there can be one daemon on the guest handling all clipboard
> events. There's some work done already by the fast-user-switch support
> and that can be extended to daemons that talk over virtio-serial.
>   

You could have one daemon that manages all vmchannel sessions.  It can 
then expose channels to apps via whatever mechanism is best.  It could 
use unix domain sockets, sys v ipc, whatever floats your boat.

And, you can build this daemon today using the existing vmchannel over 
TCP/IP.  You could also make it support serial devices.  We could also 
introduce a custom usb device and use libusb.  libusb is portable to 
Windows and Linux.

So we get backwards compatibility, and the Just Works experience with no 
funky kernel drivers.  What's there not to like?

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-07 14:14                               ` Anthony Liguori
@ 2009-08-10  6:55                                 ` Amit Shah
  2009-08-10  9:47                                   ` Gerd Hoffmann
  2009-08-10 14:27                                   ` Anthony Liguori
  0 siblings, 2 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-10  6:55 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: kvm, qemu-devel, Rusty Russell, Richard W.M. Jones,
	virtualization

On (Fri) Aug 07 2009 [09:14:43], Anthony Liguori wrote:
> Amit Shah wrote:
>> On (Thu) Aug 06 2009 [18:37:40], Jamie Lokier wrote:
>>   
>>> Apart from dbus, hard-coded meanings of small N in /dev/vmchN are
>>> asking for trouble.  It is bound to break when widely deployed and
>>>     
>>
>> It's no different from the way major-minor numbering works on the Linux
>> kernel: they uniquely identify a device.
>
> Bad example.  Quite a lot of modern devices drivers are using dynamic  
> major/minor numbers because they have proven to be such a pain in the  
> butt.  That's why we have more sophisticated mechanisms like udev for  
> userspace to make use of.

Let me explain how we came to this numbering: we first had support for
'naming' ports and the names were obtained by userspace programs by an
ioctl. Rusty suggested to use some numbering scheme where some ports
could exist at predefined locations so that we wouldn't need the naming
and the ioctls around it.

> We'll definitely need some way to support dynamic vmchannels.  Static  
> allocation of ports is just not going to work.  If we did a userspace  
> daemon, I'd suggest using some sort of universal identifier that's easy  
> to manage in a distributed fashion.  Like a reverse fqdn.
>
> So for instance, I could have an "com.ibm.my-awesome-channel" and never  
> have to worry about conflicts.

Hm, we could have something like a "class" of ports instead of the
current minor-number scheme: each port exposes what class it belongs to,
like

-virtioserial unix:/tmp/foo,class=stream -virtioserial
unix:/tmp/bar,class=console

>>> guest/host configs don't match.  It also doesn't fit comfortably when
>>> you have, say, bob and alice both logged in with desktops on separate
>>> VTs.  Clashes are inevitable, as third-party apps pick N values for
>>> themselves then get distributed - unless N values can be large
>>> (/dev/vmch44324 == kernelstats...).
>>>     
>>
>> Hm, so there can be one daemon on the guest handling all clipboard
>> events. There's some work done already by the fast-user-switch support
>> and that can be extended to daemons that talk over virtio-serial.
>>   
>
> You could have one daemon that manages all vmchannel sessions.  It can  
> then expose channels to apps via whatever mechanism is best.  It could  
> use unix domain sockets, sys v ipc, whatever floats your boat.
>
> And, you can build this daemon today using the existing vmchannel over  
> TCP/IP.  You could also make it support serial devices.  We could also  
> introduce a custom usb device and use libusb.  libusb is portable to  
> Windows and Linux.

There are some other problems with usb too: It's not transparent to
users. Any hotplug event could alert users and that's not desired. It's
a system-only thing and should also remain that way.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10  6:55                                 ` Amit Shah
@ 2009-08-10  9:47                                   ` Gerd Hoffmann
  2009-08-10 13:02                                     ` Anthony Liguori
  2009-08-10 23:09                                     ` Rusty Russell
  2009-08-10 14:27                                   ` Anthony Liguori
  1 sibling, 2 replies; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-10  9:47 UTC (permalink / raw)
  To: Amit Shah
  Cc: kvm, qemu-devel, Rusty Russell, Richard W.M. Jones,
	virtualization

On 08/10/09 08:55, Amit Shah wrote:
>> Bad example.  Quite a lot of modern devices drivers are using dynamic
>> major/minor numbers because they have proven to be such a pain in the
>> butt.  That's why we have more sophisticated mechanisms like udev for
>> userspace to make use of.
>
> Let me explain how we came to this numbering: we first had support for
> 'naming' ports and the names were obtained by userspace programs by an
> ioctl. Rusty suggested to use some numbering scheme where some ports
> could exist at predefined locations so that we wouldn't need the naming
> and the ioctls around it.

I think the naming is very important.  The guest needs to know who is 
listening on the other end of the line.  I think a sysfs attribute (as 
suggested by Jamie IIRC) will work nicely here.  So each device gets a 
property called 'class' or 'protocol' or something simliar named, 
therein a string which specifies the protocol it speaks.


Host side would look like this:

   -virtioserial port=4,protocol=clipboard

... in case the protocol is implemented in qemu or like this:

   -virtioserial port=2,protocol=libguestfs,char=unix:something

... for stuff provided by external apps.


Within the guest the two lines above would create vmch4 and vmch2, both 
having a protocol attribute, and udev then can create symlinks named by 
protocol, i.e.

   /dev/vmchannel/clipboard  symlinked to /dev/vmch4    and
   /dev/vmchannel/libguestfs symlinked to /dev/vmch2

The port=<nr> attribute can be optional and dynamically auto-allocated 
by default.


>> So for instance, I could have an "com.ibm.my-awesome-channel" and never
>> have to worry about conflicts.

reverse fqdn name space is a good idea.  We don't need a central 
protocol name registry then.  The examples above would then become 
something like this:

   protocol=orq.qemu.clipboard    and
   protocol=org.libguestfs.fish

... and within the guest

   /dev/vmchannel/org/qemu/clipboard  and
   /dev/vmchannel/org/libguestfs/fish


> There are some other problems with usb too: It's not transparent to
> users. Any hotplug event could alert users and that's not desired. It's
> a system-only thing and should also remain that way.

I think virtio-serial is the better way to handle vmchannel.  Unlike usb 
virtio is designed to work nicely in a virtual environment.

But vmchannel-over-usbserial should be easy too though in case some 
guests lacks virtio backports or something.  I think you can just stick 
a name like "vmchannel:orq.qemu.clipboard" into the usbserial product 
name, then have udev match that and create
   /dev/vmchannel/org/qemu/clipboard symlinking to /dev/ttyUSB<nr>

Voila, you can switch transports and the apps don't even notice.

cheers,
   Gerd

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10  9:47                                   ` Gerd Hoffmann
@ 2009-08-10 13:02                                     ` Anthony Liguori
  2009-08-10 14:02                                       ` Gerd Hoffmann
  2009-08-10 23:09                                     ` Rusty Russell
  1 sibling, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-10 13:02 UTC (permalink / raw)
  To: Gerd Hoffmann
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

Gerd Hoffmann wrote:
>
>> There are some other problems with usb too: It's not transparent to
>> users. Any hotplug event could alert users and that's not desired. It's
>> a system-only thing and should also remain that way.
>
> I think virtio-serial is the better way to handle vmchannel.  Unlike 
> usb virtio is designed to work nicely in a virtual environment.
>
> But vmchannel-over-usbserial should be easy too though in case some 
> guests lacks virtio backports or something.

I think you're missing my fundamental point.  Don't use the kernel as 
the guest interface.

Introduce a userspace daemon that exposes a domain socket.  Then we can 
have a proper protocol that uses reverse fqdns for identification.

We can do the backend over TCP/IP, usb, standard serial, etc.

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 13:02                                     ` Anthony Liguori
@ 2009-08-10 14:02                                       ` Gerd Hoffmann
  2009-08-10 14:20                                         ` Anthony Liguori
  2009-08-10 14:20                                         ` Anthony Liguori
  0 siblings, 2 replies; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-10 14:02 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

On 08/10/09 15:02, Anthony Liguori wrote:

> I think you're missing my fundamental point. Don't use the kernel as the
> guest interface.
>
> Introduce a userspace daemon that exposes a domain socket. Then we can
> have a proper protocol that uses reverse fqdns for identification.

We need nothing but (a) bidirectional byte streams and (b) name tags for 
them.

Do we really want design a daemon and a protocol for such a simple 
thing?  Especially as requiring a daemon for that adds a few problems 
you don't have without them.  Access control for example:  For device 
nodes you can just use standard unix permissions and acls.  You can 
easily do stuff like adding the logged in desktop user to the 
/dev/vmchannel/org/qemu/clipboard acl using existing solutions.  With a 
daemon you have to hop through a number of loops to archive the same.

Can't we simply have guest apps open "/dev/vmchannel/$protocol" ?

cheers,
   Gerd

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 14:02                                       ` Gerd Hoffmann
@ 2009-08-10 14:20                                         ` Anthony Liguori
  2009-08-10 15:34                                           ` Gerd Hoffmann
  2009-08-10 14:20                                         ` Anthony Liguori
  1 sibling, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-10 14:20 UTC (permalink / raw)
  To: Gerd Hoffmann
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

Gerd Hoffmann wrote:
> On 08/10/09 15:02, Anthony Liguori wrote:
>
>> I think you're missing my fundamental point. Don't use the kernel as the
>> guest interface.
>>
>> Introduce a userspace daemon that exposes a domain socket. Then we can
>> have a proper protocol that uses reverse fqdns for identification.
>
> We need nothing but (a) bidirectional byte streams and (b) name tags 
> for them.
>
> Do we really want design a daemon and a protocol for such a simple thing?

Yes, because we also need (c) the ability to write cross platform 
software that targets vmchannel.

So having a library interface is going to be extremely desirable.

Also, see the previous discussion about security.  How do you sanely 
delegate /dev/vmchannel/org/qemu/clipboard to the current Xorg user?

>   Especially as requiring a daemon for that adds a few problems you 
> don't have without them.  Access control for example:  For device 
> nodes you can just use standard unix permissions and acls.

But how do you set those permissions in the first place?

>   You can easily do stuff like adding the logged in desktop user to 
> the /dev/vmchannel/org/qemu/clipboard acl using existing solutions.  
> With a daemon you have to hop through a number of loops to archive the 
> same.
>
> Can't we simply have guest apps open "/dev/vmchannel/$protocol" ?

/dev interfaces are only simple to kernel developers :-)  Besides, why 
do something that can be clearly done in userspace within the kernel?  
It just increases the possibility of kernel bugs.

You can have a /var/run/vmchannel/$protocol.sock unix domain socket and 
it has all the same properties that you describe.  It also Just Works 
with standard tools like socat.  You can transparently route it over the 
network, have it work over slirp, a serial device, or some custom virtio 
device if we so choose.

It's the only sane way to support older guests too.  If we really want 
vmchannel to be used by application developers, then we really need a 
libvmchannel.

Regards,

Anthony Liguori

> cheers,
>   Gerd
>

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 14:02                                       ` Gerd Hoffmann
  2009-08-10 14:20                                         ` Anthony Liguori
@ 2009-08-10 14:20                                         ` Anthony Liguori
  1 sibling, 0 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-08-10 14:20 UTC (permalink / raw)
  To: Gerd Hoffmann
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

Gerd Hoffmann wrote:
> On 08/10/09 15:02, Anthony Liguori wrote:
>
>> I think you're missing my fundamental point. Don't use the kernel as the
>> guest interface.
>>
>> Introduce a userspace daemon that exposes a domain socket. Then we can
>> have a proper protocol that uses reverse fqdns for identification.
>
> We need nothing but (a) bidirectional byte streams and (b) name tags 
> for them.
>
> Do we really want design a daemon and a protocol for such a simple thing?

Yes, because we also need (c) the ability to write cross platform 
software that targets vmchannel.

So having a library interface is going to be extremely desirable.

Also, see the previous discussion about security.  How do you sanely 
delegate /dev/vmchannel/org/qemu/clipboard to the current Xorg user?

>   Especially as requiring a daemon for that adds a few problems you 
> don't have without them.  Access control for example:  For device 
> nodes you can just use standard unix permissions and acls.

But how do you set those permissions in the first place?

>   You can easily do stuff like adding the logged in desktop user to 
> the /dev/vmchannel/org/qemu/clipboard acl using existing solutions.  
> With a daemon you have to hop through a number of loops to archive the 
> same.
>
> Can't we simply have guest apps open "/dev/vmchannel/$protocol" ?

/dev interfaces are only simple to kernel developers :-)  Besides, why 
do something that can be clearly done in userspace within the kernel?  
It just increases the possibility of kernel bugs.

You can have a /var/run/vmchannel/$protocol.sock unix domain socket and 
it has all the same properties that you describe.  It also Just Works 
with standard tools like socat.  You can transparently route it over the 
network, have it work over slirp, a serial device, or some custom virtio 
device if we so choose.

It's the only sane way to support older guests too.  If we really want 
vmchannel to be used by application developers, then we really need a 
libvmchannel.

Regards,

Anthony Liguori

> cheers,
>   Gerd
>

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10  6:55                                 ` Amit Shah
  2009-08-10  9:47                                   ` Gerd Hoffmann
@ 2009-08-10 14:27                                   ` Anthony Liguori
  2009-08-10 15:57                                     ` Gerd Hoffmann
  1 sibling, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-10 14:27 UTC (permalink / raw)
  To: Amit Shah
  Cc: kvm, qemu-devel, Rusty Russell, Richard W.M. Jones,
	virtualization

Amit Shah wrote:
> Let me explain how we came to this numbering: we first had support for
> 'naming' ports and the names were obtained by userspace programs by an
> ioctl. Rusty suggested to use some numbering scheme where some ports
> could exist at predefined locations so that we wouldn't need the naming
> and the ioctls around it.
>   

Fortunately, if you implement the naming scheme in userspace you get the 
best of both worlds ;-)

>>> Hm, so there can be one daemon on the guest handling all clipboard
>>> events. There's some work done already by the fast-user-switch support
>>> and that can be extended to daemons that talk over virtio-serial.
>>>   
>>>       
>> You could have one daemon that manages all vmchannel sessions.  It can  
>> then expose channels to apps via whatever mechanism is best.  It could  
>> use unix domain sockets, sys v ipc, whatever floats your boat.
>>
>> And, you can build this daemon today using the existing vmchannel over  
>> TCP/IP.  You could also make it support serial devices.  We could also  
>> introduce a custom usb device and use libusb.  libusb is portable to  
>> Windows and Linux.
>>     
>
> There are some other problems with usb too: It's not transparent to
> users. Any hotplug event could alert users and that's not desired.

I don't think this is true in practice.  Our goal is not to circumvent 
an OS's policy decisions either.

>  It's
> a system-only thing and should also remain that way.
>   

I don't buy this argument at all.  If you exposed a new usb device that 
no OS had a kernel driver, and you had a daemon running that watched for 
insertions of that device, what OS would that not work transparently on?

I think my fundamental argument boils down to two points.  1) we should 
not require new guest drivers unless we absolutely have to 2) we should 
always do things in userspace unless we absolutely have to do things in 
the kernel.

Adding new kernel drivers breaks support for enterprise Linux distros.  
Adding a userspace daemon does not.  Windows device drivers require 
signing which is very difficult to do.  There's a huge practical 
advantage in not requiring guest drivers.

Regards,

Anthony Liguori

> 		Amit
>   

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 14:20                                         ` Anthony Liguori
@ 2009-08-10 15:34                                           ` Gerd Hoffmann
  2009-08-10 16:59                                             ` Anthony Liguori
  0 siblings, 1 reply; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-10 15:34 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

On 08/10/09 16:20, Anthony Liguori wrote:
> Gerd Hoffmann wrote:
>> Do we really want design a daemon and a protocol for such a simple thing?
>
> Yes, because we also need (c) the ability to write cross platform
> software that targets vmchannel.
>
> So having a library interface is going to be extremely desirable.

You don't need a daemon for that though.

> Also, see the previous discussion about security. How do you sanely
> delegate /dev/vmchannel/org/qemu/clipboard to the current Xorg user?

pam_console (I think that is the name of the beast).
Or is it handled by hal these days?

The piece of software which does the very same thing already for sound 
and other devices.

>> Especially as requiring a daemon for that adds a few problems you
>> don't have without them. Access control for example: For device nodes
>> you can just use standard unix permissions and acls.
>
> But how do you set those permissions in the first place?

See above.  There are other devices which need that too.  There are 
existing solutions for this problem.

>> You can easily do stuff like adding the logged in desktop user to the
>> /dev/vmchannel/org/qemu/clipboard acl using existing solutions. With a
>> daemon you have to hop through a number of loops to archive the same.
>>
>> Can't we simply have guest apps open "/dev/vmchannel/$protocol" ?
>
> /dev interfaces are only simple to kernel developers :-) Besides, why do
> something that can be clearly done in userspace within the kernel?

Ok, lets rip out the in-kernel ioapic code then.  It can (and has been) 
done in userspace.

> It
> just increases the possibility of kernel bugs.

The daemon increases the possibility of userspace bugs.

Seriously:  Attaching a name tag to virtio-serial devices and have them 
exposed via sysfs is probably *much* less code than a vmchannel daemon.

Also multiplexing over one device introduces a number of problems you 
have to take care of on both sides (qemu+daemon) of the connection.  For 
example:  When the communication stalls in one protocol the others 
should keep on going of course.  With one device per protocol and thus 
one virtqueue per protocol the problem doesn't exist in the first place.

> You can have a /var/run/vmchannel/$protocol.sock unix domain socket and
> it has all the same properties that you describe. It also Just Works
> with standard tools like socat.

bash: socat: command not found

> If we really want
> vmchannel to be used by application developers, then we really need a
> libvmchannel.

We need a sane solution developed and merged and not a new idea each week.

cheers,
   Gerd

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 14:27                                   ` Anthony Liguori
@ 2009-08-10 15:57                                     ` Gerd Hoffmann
  0 siblings, 0 replies; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-10 15:57 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

On 08/10/09 16:27, Anthony Liguori wrote:

> I think my fundamental argument boils down to two points. 1) we should
> not require new guest drivers unless we absolutely have to

Allow guest drivers is fine though I guess?

> 2) we should
> always do things in userspace unless we absolutely have to do things in
> the kernel.

Wrong.  There are often good reasons to do stuff in kernel, even if you 
can do it in userspace too.

> Adding new kernel drivers breaks support for enterprise Linux distros.
> Adding a userspace daemon does not. Windows device drivers require
> signing which is very difficult to do. There's a huge practical
> advantage in not requiring guest drivers.

Ok, so the virtio-serial + usbserial combo should work well then I think.

If you have guest drivers you'll go the virtio-serial route.
If you don't have guest drivers you can go the usbserial route, either 
via /dev/ttyUSB or via libusb.

We can also have a libvmchannel as abstraction layer on top of this.

cheers,
   Gerd

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 15:34                                           ` Gerd Hoffmann
@ 2009-08-10 16:59                                             ` Anthony Liguori
  2009-08-10 17:27                                               ` Anthony Liguori
                                                                 ` (2 more replies)
  0 siblings, 3 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-08-10 16:59 UTC (permalink / raw)
  To: Gerd Hoffmann
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

Gerd Hoffmann wrote:
> Ok, lets rip out the in-kernel ioapic code then.  It can (and has 
> been) done in userspace.

The justification is performance although that's not really true anymore 
post TPR optimization.

But FWIW, I opposed both the in-kernel apic and the in-kernel pit when 
they were introduced.  If nothing else, I'm at least consistent :-)

> The daemon increases the possibility of userspace bugs.

How?

> Seriously:  Attaching a name tag to virtio-serial devices and have 
> them exposed via sysfs is probably *much* less code than a vmchannel 
> daemon.

I strongly doubt that.

>> If we really want
>> vmchannel to be used by application developers, then we really need a
>> libvmchannel.
>
> We need a sane solution developed and merged and not a new idea each 
> week.

There is nothing sane about vmchannel.  It's just an attempt to bypass 
QEMU which is going to introduce all sorts of complexities wrt 
migration, guest compatibility, etc.

However, as I've mentioned repeatedly, the reason I won't merge 
virtio-serial is that it duplicates functionality with virtio-console.  
If the two are converged, I'm happy to merge it.  I'm not opposed to 
having more functionality.

I think it's the wrong solution for the use-case, and I always have, but 
that's independent of my willingness to merge it.

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 16:59                                             ` Anthony Liguori
@ 2009-08-10 17:27                                               ` Anthony Liguori
  2009-08-12 18:27                                               ` Paul Brook
  2009-08-14  8:15                                               ` Amit Shah
  2 siblings, 0 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-08-10 17:27 UTC (permalink / raw)
  To: Gerd Hoffmann
  Cc: kvm, Richard W.M. Jones, Rusty Russell, qemu-devel,
	virtualization, Amit Shah

Anthony Liguori wrote:
>
> There is nothing sane about vmchannel.  It's just an attempt to bypass 
> QEMU which is going to introduce all sorts of complexities wrt 
> migration, guest compatibility, etc.
>
> However, as I've mentioned repeatedly, the reason I won't merge 
> virtio-serial is that it duplicates functionality with 
> virtio-console.  If the two are converged, I'm happy to merge it.  I'm 
> not opposed to having more functionality.

NB: the userspace interface for these devices should be a tty, not a new 
character device.

If you want to add a new bustype for these devices, and then have an 
entry in sysfs that had some sort of identification string, that's 
perfectly acceptable.

Also note though that this is exactly what usb-serial is today.  
/sys/bus/usbserial contains all the usb serial devices and you can get a 
vendor id/device id to uniquely identify the device type.

Using virtio vs. usb has it's advantage but the userspace interface 
model should be roughly equivalent.

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10  9:47                                   ` Gerd Hoffmann
  2009-08-10 13:02                                     ` Anthony Liguori
@ 2009-08-10 23:09                                     ` Rusty Russell
  2009-08-11  0:16                                       ` Anthony Liguori
  1 sibling, 1 reply; 62+ messages in thread
From: Rusty Russell @ 2009-08-10 23:09 UTC (permalink / raw)
  To: Gerd Hoffmann
  Cc: kvm, qemu-devel, Richard W.M. Jones, virtualization, Amit Shah

On Mon, 10 Aug 2009 07:17:54 pm Gerd Hoffmann wrote:
> On 08/10/09 08:55, Amit Shah wrote:
> >> Bad example.  Quite a lot of modern devices drivers are using dynamic
> >> major/minor numbers because they have proven to be such a pain in the
> >> butt.  That's why we have more sophisticated mechanisms like udev for
> >> userspace to make use of.
> >
> > Let me explain how we came to this numbering: we first had support for
> > 'naming' ports and the names were obtained by userspace programs by an
> > ioctl. Rusty suggested to use some numbering scheme where some ports
> > could exist at predefined locations so that we wouldn't need the naming
> > and the ioctls around it.
> 
> I think the naming is very important.

I disagree.  If you can hand out names, you can hand out numbers.  Whether
the guest chooses to put that number in sysfs or make it a minor, I don't
care.

Rusty.

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 23:09                                     ` Rusty Russell
@ 2009-08-11  0:16                                       ` Anthony Liguori
  0 siblings, 0 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-08-11  0:16 UTC (permalink / raw)
  To: Rusty Russell
  Cc: kvm, qemu-devel, Richard W.M. Jones, virtualization,
	Gerd Hoffmann, Amit Shah

Rusty Russell wrote:
> On Mon, 10 Aug 2009 07:17:54 pm Gerd Hoffmann wrote:
>   
>> On 08/10/09 08:55, Amit Shah wrote:
>>     
>>>> Bad example.  Quite a lot of modern devices drivers are using dynamic
>>>> major/minor numbers because they have proven to be such a pain in the
>>>> butt.  That's why we have more sophisticated mechanisms like udev for
>>>> userspace to make use of.
>>>>         
>>> Let me explain how we came to this numbering: we first had support for
>>> 'naming' ports and the names were obtained by userspace programs by an
>>> ioctl. Rusty suggested to use some numbering scheme where some ports
>>> could exist at predefined locations so that we wouldn't need the naming
>>> and the ioctls around it.
>>>       
>> I think the naming is very important.
>>     
>
> I disagree.  If you can hand out names, you can hand out numbers.

The problem with handing out names is that there has to be someone to 
"hand" things out.  And even if you have a good hander-outer, 
development is difficult in a distributed environment because you may 
have folks using your code before you've gotten an official hand-out.

A better discovery mechanism is based on something that piggy backs on 
another authority.  For instance, reverse fully qualified domains work 
well.  uuid's tend to work pretty well too although it's not perfect.

In general, even just open strings can work out okay given that people 
are responsible in how they name things.

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 16:59                                             ` Anthony Liguori
  2009-08-10 17:27                                               ` Anthony Liguori
@ 2009-08-12 18:27                                               ` Paul Brook
  2009-08-14  8:15                                               ` Amit Shah
  2 siblings, 0 replies; 62+ messages in thread
From: Paul Brook @ 2009-08-12 18:27 UTC (permalink / raw)
  To: qemu-devel
  Cc: kvm, Rusty Russell, Richard W.M. Jones, virtualization,
	Gerd Hoffmann, Amit Shah

> However, as I've mentioned repeatedly, the reason I won't merge
> virtio-serial is that it duplicates functionality with virtio-console.
> If the two are converged, I'm happy to merge it.  I'm not opposed to
> having more functionality.

I strongly agree.

Paul

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-10 16:59                                             ` Anthony Liguori
  2009-08-10 17:27                                               ` Anthony Liguori
  2009-08-12 18:27                                               ` Paul Brook
@ 2009-08-14  8:15                                               ` Amit Shah
  2009-08-14 13:29                                                 ` Anthony Liguori
  2009-08-14 13:49                                                 ` Gerd Hoffmann
  2 siblings, 2 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-14  8:15 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: Rusty Russell, virtualization, Gerd Hoffmann, kvm, qemu-devel

On (Mon) Aug 10 2009 [11:59:31], Anthony Liguori wrote:
>
> However, as I've mentioned repeatedly, the reason I won't merge  
> virtio-serial is that it duplicates functionality with virtio-console.   
> If the two are converged, I'm happy to merge it.  I'm not opposed to  
> having more functionality.

The guest code sort-of ends up looking like this after merging
virtio_console into virtio_serial. Diff is against virtio_serial in my
git tree, but that should be pretty close to the last submission I made
at

http://patchwork.kernel.org/patch/39335/

or the tree at

git://git.kernel.org/pub/scm/linux/kernel/git/amit/vs-kernel.git

I've merged bits from virtio_console.c into virtio_serial.c. If needed,
virtio_serial can be renamed to virtio_console. The VIRITIO_ID also
needs to change to that of virtio_console's.

Similar changes are needed for userspace.

Since there's support for only one console as of now, I've assigned port
#2 as the console port so that we hook into hvc when a port is found at
that location.

One issue that crops up for put_chars: a copy of the buffer to be sent
has to be made as we don't wait for host to ack the buffer before we
move on.

The biggest loss so far is Rusty's excellent comments: they will have to
be reworked and added for the whole of the new file.

Is this approach acceptable?


diff --git a/drivers/char/Makefile b/drivers/char/Makefile
index 5e1915b..ab9c914 100644
--- a/drivers/char/Makefile
+++ b/drivers/char/Makefile
@@ -53,7 +53,6 @@ obj-$(CONFIG_HVC_IRQ)		+= hvc_irq.o
 obj-$(CONFIG_HVC_XEN)		+= hvc_xen.o
 obj-$(CONFIG_HVC_IUCV)		+= hvc_iucv.o
 obj-$(CONFIG_HVC_UDBG)		+= hvc_udbg.o
-obj-$(CONFIG_VIRTIO_CONSOLE)	+= virtio_console.o
 obj-$(CONFIG_VIRTIO_SERIAL)	+= virtio_serial.o
 obj-$(CONFIG_RAW_DRIVER)	+= raw.o
 obj-$(CONFIG_SGI_SNSC)		+= snsc.o snsc_event.o
diff --git a/drivers/char/virtio_console.c b/drivers/char/virtio_console.c
index c74dacf..f82c036 100644
--- a/drivers/char/virtio_console.c
+++ b/drivers/char/virtio_console.c
@@ -43,39 +43,6 @@ static struct virtio_device *vdev;
 static unsigned int in_len;
 static char *in, *inbuf;
 
-/* The operations for our console. */
-static struct hv_ops virtio_cons;
-
-/* The hvc device */
-static struct hvc_struct *hvc;
-
-/*D:310 The put_chars() callback is pretty straightforward.
- *
- * We turn the characters into a scatter-gather list, add it to the output
- * queue and then kick the Host.  Then we sit here waiting for it to finish:
- * inefficient in theory, but in practice implementations will do it
- * immediately (lguest's Launcher does). */
-static int put_chars(u32 vtermno, const char *buf, int count)
-{
-	struct scatterlist sg[1];
-	unsigned int len;
-
-	/* This is a convenient routine to initialize a single-elem sg list */
-	sg_init_one(sg, buf, count);
-
-	/* add_buf wants a token to identify this buffer: we hand it any
-	 * non-NULL pointer, since there's only ever one buffer. */
-	if (out_vq->vq_ops->add_buf(out_vq, sg, 1, 0, (void *)1) == 0) {
-		/* Tell Host to go! */
-		out_vq->vq_ops->kick(out_vq);
-		/* Chill out until it's done with the buffer. */
-		while (!out_vq->vq_ops->get_buf(out_vq, &len))
-			cpu_relax();
-	}
-
-	/* We're expected to return the amount of data we wrote: all of it. */
-	return count;
-}
 
 /* Create a scatter-gather list representing our input buffer and put it in the
  * queue. */
@@ -90,94 +57,7 @@ static void add_inbuf(void)
 	in_vq->vq_ops->kick(in_vq);
 }
 
-/*D:350 get_chars() is the callback from the hvc_console infrastructure when
- * an interrupt is received.
- *
- * Most of the code deals with the fact that the hvc_console() infrastructure
- * only asks us for 16 bytes at a time.  We keep in_offset and in_used fields
- * for partially-filled buffers. */
-static int get_chars(u32 vtermno, char *buf, int count)
-{
-	/* If we don't have an input queue yet, we can't get input. */
-	BUG_ON(!in_vq);
-
-	/* No buffer?  Try to get one. */
-	if (!in_len) {
-		in = in_vq->vq_ops->get_buf(in_vq, &in_len);
-		if (!in)
-			return 0;
-	}
-
-	/* You want more than we have to give?  Well, try wanting less! */
-	if (in_len < count)
-		count = in_len;
-
-	/* Copy across to their buffer and increment offset. */
-	memcpy(buf, in, count);
-	in += count;
-	in_len -= count;
 
-	/* Finished?  Re-register buffer so Host will use it again. */
-	if (in_len == 0)
-		add_inbuf();
-
-	return count;
-}
-/*:*/
-
-/*D:320 Console drivers are initialized very early so boot messages can go out,
- * so we do things slightly differently from the generic virtio initialization
- * of the net and block drivers.
- *
- * At this stage, the console is output-only.  It's too early to set up a
- * virtqueue, so we let the drivers do some boutique early-output thing. */
-int __init virtio_cons_early_init(int (*put_chars)(u32, const char *, int))
-{
-	virtio_cons.put_chars = put_chars;
-	return hvc_instantiate(0, 0, &virtio_cons);
-}
-
-/*
- * virtio console configuration. This supports:
- * - console resize
- */
-static void virtcons_apply_config(struct virtio_device *dev)
-{
-	struct winsize ws;
-
-	if (virtio_has_feature(dev, VIRTIO_CONSOLE_F_SIZE)) {
-		dev->config->get(dev,
-				 offsetof(struct virtio_console_config, cols),
-				 &ws.ws_col, sizeof(u16));
-		dev->config->get(dev,
-				 offsetof(struct virtio_console_config, rows),
-				 &ws.ws_row, sizeof(u16));
-		hvc_resize(hvc, ws);
-	}
-}
-
-/*
- * we support only one console, the hvc struct is a global var
- * We set the configuration at this point, since we now have a tty
- */
-static int notifier_add_vio(struct hvc_struct *hp, int data)
-{
-	hp->irq_requested = 1;
-	virtcons_apply_config(vdev);
-
-	return 0;
-}
-
-static void notifier_del_vio(struct hvc_struct *hp, int data)
-{
-	hp->irq_requested = 0;
-}
-
-static void hvc_handle_input(struct virtqueue *vq)
-{
-	if (hvc_poll(hvc))
-		hvc_kick();
-}
 
 /*D:370 Once we're further in boot, we get probed like any other virtio device.
  * At this stage we set up the output virtqueue.
@@ -212,27 +92,7 @@ static int __devinit virtcons_probe(struct virtio_device *dev)
 	in_vq = vqs[0];
 	out_vq = vqs[1];
 
-	/* Start using the new console output. */
-	virtio_cons.get_chars = get_chars;
-	virtio_cons.put_chars = put_chars;
-	virtio_cons.notifier_add = notifier_add_vio;
-	virtio_cons.notifier_del = notifier_del_vio;
-	virtio_cons.notifier_hangup = notifier_del_vio;
 
-	/* The first argument of hvc_alloc() is the virtual console number, so
-	 * we use zero.  The second argument is the parameter for the
-	 * notification mechanism (like irq number). We currently leave this
-	 * as zero, virtqueues have implicit notifications.
-	 *
-	 * The third argument is a "struct hv_ops" containing the put_chars()
-	 * get_chars(), notifier_add() and notifier_del() pointers.
-	 * The final argument is the output buffer size: we can do any size,
-	 * so we put PAGE_SIZE here. */
-	hvc = hvc_alloc(0, 0, &virtio_cons, PAGE_SIZE);
-	if (IS_ERR(hvc)) {
-		err = PTR_ERR(hvc);
-		goto free_vqs;
-	}
 
 	/* Register the input buffer the first time. */
 	add_inbuf();
diff --git a/drivers/char/virtio_serial.c b/drivers/char/virtio_serial.c
index ef2d730..ff6ad06 100644
--- a/drivers/char/virtio_serial.c
+++ b/drivers/char/virtio_serial.c
@@ -37,6 +37,7 @@
 #include <linux/virtio.h>
 #include <linux/virtio_serial.h>
 #include <linux/workqueue.h>
+#include "hvc_console.h"
 
 struct virtio_serial_struct {
 	struct work_struct rx_work;
@@ -131,27 +132,13 @@ static int send_control_event(struct virtio_serial_control *sercontrol)
 	return ret;
 }
 
-static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
-			       size_t count, loff_t *offp)
+static ssize_t fill_readbuf(struct virtio_serial_port *port, char *out_buf,
+			    size_t count, bool to_user)
 {
-	struct virtio_serial_port *port;
 	struct virtio_serial_port_buffer *buf, *buf2;
-	ssize_t ubuf_offset, ret;
-
-	port = filp->private_data;
-
-	ret = 0;
-	if (list_empty(&port->readbuf_head)) {
-		if (filp->f_flags & O_NONBLOCK)
-			return -EAGAIN;
+	ssize_t out_offset, ret;
 
-		ret = wait_event_interruptible(port->waitqueue,
-					       !list_empty(&port->readbuf_head));
-	}
-	if (ret < 0)
-		return ret;
-
-	ubuf_offset = 0;
+	out_offset = 0;
 	list_for_each_entry_safe(buf, buf2, &port->readbuf_head, next) {
 		size_t copy_size;
 
@@ -159,16 +146,24 @@ static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
 		if (copy_size > buf->len - buf->offset)
 			copy_size = buf->len - buf->offset;
 
-		ret = copy_to_user(ubuf + ubuf_offset, buf->buf + buf->offset,
-				   copy_size);
+		if (to_user) {
+			ret = copy_to_user(out_buf + out_offset,
+					   buf->buf + buf->offset,
+					   copy_size);
+			/* FIXME: Deal with ret != 0 */
+		} else {
+			memcpy(out_buf + out_offset,
+			       buf->buf + buf->offset,
+			       copy_size);
+			ret = 0; /* Emulate copy_to_user behaviour */
+		}
 
-		/* FIXME: Deal with ret != 0 */
 		/* Return the number of bytes actually copied */
 		ret = copy_size - ret;
 		buf->offset += ret;
-		ubuf_offset += ret;
+		out_offset += ret;
 		count -= ret;
-		ret = ubuf_offset;
+		ret = out_offset;
 
 		if (buf->len - buf->offset == 0) {
 			list_del(&buf->next);
@@ -178,6 +173,30 @@ static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
 		if (!count)
 			break;
 	}
+	return out_offset;
+}
+
+static ssize_t virtserial_read(struct file *filp, char __user *ubuf,
+			       size_t count, loff_t *offp)
+{
+	struct virtio_serial_port *port;
+	ssize_t ret;
+
+	port = filp->private_data;
+
+	ret = 0;
+	if (list_empty(&port->readbuf_head)) {
+		if (filp->f_flags & O_NONBLOCK)
+			return -EAGAIN;
+
+		ret = wait_event_interruptible(port->waitqueue,
+					       !list_empty(&port->readbuf_head));
+	}
+	if (ret < 0)
+		return ret;
+
+	ret = fill_readbuf(port, ubuf, count, 1);
+
 	return ret;
 }
 
@@ -196,21 +215,19 @@ struct vbuf {
 	unsigned int nent;
 };
 
-static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
-				size_t count, loff_t *offp)
+static ssize_t send_writebuf(struct virtio_serial_port *port,
+			     const char *in_buf, size_t count, bool from_user)
 {
 	struct virtqueue *out_vq;
-	struct virtio_serial_port *port;
 	struct virtio_serial_id id;
 	struct vbuf *vbuf;
-	size_t offset, size;
+	size_t in_offset, copy_size;
 	ssize_t ret;
 	unsigned int i, id_len;
 
 	if (!count)
 		return 0;
 
-	port = filp->private_data;
 	id.id = get_id_from_port(port);
 	out_vq = virtserial.out_vq;
 
@@ -234,10 +251,10 @@ static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
 	sg_init_table(vbuf->sg, vbuf->nent);
 
 	i = 0; /* vbuf->bufs[i] */
-	offset = 0; /* offset in the user buffer */
-	while (count - offset) {
-		size = min(count - offset + id_len, PAGE_SIZE);
-		vbuf->bufs[i] = kzalloc(size, GFP_KERNEL);
+	in_offset = 0; /* offset in the source buffer */
+	while (count - in_offset) {
+		copy_size = min(count - in_offset + id_len, PAGE_SIZE);
+		vbuf->bufs[i] = kzalloc(copy_size, GFP_KERNEL);
 		if (!vbuf->bufs[i]) {
 			ret = -ENOMEM;
 			if (!i)
@@ -246,12 +263,25 @@ static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
 		}
 		if (id_len) {
 			memcpy(vbuf->bufs[i], &id, id_len);
-			size -= id_len;
+			copy_size -= id_len;
 		}
-		ret = copy_from_user(vbuf->bufs[i] + id_len, ubuf + offset, size);
-		offset += size - ret;
+		if (from_user)
+			ret = copy_from_user(vbuf->bufs[i] + id_len,
+					     in_buf + in_offset, copy_size);
+		else {
+			/* Since we're not sure when the host will actually
+			 * consume the data and tell us about it, we have
+			 * to copy the data here in case the caller
+			 * frees the string
+			 */
+			memcpy(vbuf->bufs[i] + id_len,
+			       in_buf + in_offset, copy_size);
+			ret = 0; /* Emulate copy_from_user */
+		}
+		in_offset += copy_size - ret;
 
-		sg_set_buf(&vbuf->sg[i], vbuf->bufs[i], size - ret + id_len);
+		sg_set_buf(&vbuf->sg[i], vbuf->bufs[i],
+			   copy_size - ret + id_len);
 		id_len = 0; /* Pass the port id only in the first buffer */
 		i++;
 	}
@@ -263,7 +293,7 @@ static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
 	out_vq->vq_ops->kick(out_vq);
 
 	/* We're expected to return the amount of data we wrote */
-	return offset;
+	return in_offset;
 free_buffers:
 	while (--i >= 0)
 		kfree(vbuf->bufs[i]);
@@ -276,6 +306,16 @@ free_vbuf:
 	return ret;
 }
 
+static ssize_t virtserial_write(struct file *filp, const char __user *ubuf,
+				size_t count, loff_t *offp)
+{
+	struct virtio_serial_port *port;
+
+	port = filp->private_data;
+
+	return send_writebuf(port, ubuf, count, 1);
+}
+
 static int virtserial_release(struct inode *inode, struct file *filp)
 {
 	struct virtio_serial_control *sercontrol;
@@ -343,6 +383,104 @@ static const struct file_operations virtserial_fops = {
 	.release = virtserial_release,
 };
 
+/* Some routines for supporting virtio console */
+
+/* The operations for our console. */
+static struct hv_ops virtio_cons;
+
+/* The hvc device */
+static struct hvc_struct *hvc;
+
+/*D:310 The console_put_chars() callback is pretty straightforward.
+ *
+ * We turn the characters into a scatter-gather list, add it to the output
+ * queue and then kick the Host.  Then we sit here waiting for it to finish:
+ * inefficient in theory, but in practice implementations will do it
+ * immediately (lguest's Launcher does). */
+static int console_put_chars(u32 vtermno, const char *buf, int count)
+{
+	struct virtio_serial_port *port;
+
+	port = get_port_from_id(VIRTIO_SERIAL_CONSOLE_PORT);
+	if (!port)
+		return 0;
+
+	return send_writebuf(port, buf, count, 0);
+}
+
+/*D:350 console_get_chars() is the callback from the hvc_console
+ * infrastructure when an interrupt is received.
+ *
+ * Most of the code deals with the fact that the hvc_console() infrastructure
+ * only asks us for 16 bytes at a time.  We keep in_offset and in_used fields
+ * for partially-filled buffers. */
+static int console_get_chars(u32 vtermno, char *out_buf, int count)
+{
+	struct virtio_serial_port *port;
+
+	/* If we don't have an input queue yet, we can't get input. */
+	BUG_ON(!virtserial.in_vq);
+
+	port = get_port_from_id(VIRTIO_SERIAL_CONSOLE_PORT);
+	if (!port)
+		return 0;
+
+	if (list_empty(&port->readbuf_head))
+		return 0;
+
+	return fill_readbuf(port, out_buf, count, 0);
+}
+/*:*/
+
+/*D:320 Console drivers are initialized very early so boot messages can go out,
+ * so we do things slightly differently from the generic virtio initialization
+ * of the net and block drivers.
+ *
+ * At this stage, the console is output-only.  It's too early to set up a
+ * virtqueue, so we let the drivers do some boutique early-output thing. */
+int __init virtio_cons_early_init(int (*put_chars)(u32, const char *, int))
+{
+	virtio_cons.put_chars = put_chars;
+	return hvc_instantiate(0, 0, &virtio_cons);
+}
+
+/*
+ * virtio console configuration. This supports:
+ * - console resize
+ */
+static void virtcons_apply_config(struct virtio_device *dev)
+{
+	struct winsize ws;
+
+	if (virtio_has_feature(dev, VIRTIO_CONSOLE_F_SIZE)) {
+		dev->config->get(dev,
+				 offsetof(struct virtio_serial_config, cols),
+				 &ws.ws_col, sizeof(u16));
+		dev->config->get(dev,
+				 offsetof(struct virtio_serial_config, rows),
+				 &ws.ws_row, sizeof(u16));
+		hvc_resize(hvc, ws);
+	}
+}
+
+/*
+ * we support only one console, the hvc struct is a global var
+ * We set the configuration at this point, since we now have a tty
+ */
+static int console_notifier_add_vio(struct hvc_struct *hp, int data)
+{
+	hp->irq_requested = 1;
+	virtcons_apply_config(virtserial.vdev);
+
+	return 0;
+}
+
+static void console_notifier_del_vio(struct hvc_struct *hp, int data)
+{
+	hp->irq_requested = 0;
+}
+
+
 static void virtio_serial_queue_work_handler(struct work_struct *work)
 {
 	struct scatterlist sg[1];
@@ -410,6 +548,10 @@ static void virtio_serial_rx_work_handler(struct work_struct *work)
 
 		wake_up_interruptible(&port->waitqueue);
 	}
+
+	if (get_id_from_port(port) == VIRTIO_SERIAL_CONSOLE_PORT && hvc_poll(hvc))
+		hvc_kick();
+
 	/* Allocate buffers for all the ones that got used up */
 	schedule_work(&virtserial.queue_work);
 }
@@ -555,6 +697,33 @@ static int virtserial_add_port(u32 port_nr)
 
 	list_add_tail(&port->next, &virtserial.port_head);
 
+	if (port_nr == VIRTIO_SERIAL_CONSOLE_PORT) {
+		/* Start using the new console output. */
+		virtio_cons.get_chars = console_get_chars;
+		virtio_cons.put_chars = console_put_chars;
+		virtio_cons.notifier_add = console_notifier_add_vio;
+		virtio_cons.notifier_del = console_notifier_del_vio;
+		virtio_cons.notifier_hangup = console_notifier_del_vio;
+
+		/* The first argument of hvc_alloc() is the virtual
+		 * console number, so we use zero.  The second
+		 * argument is the parameter for the notification
+		 * mechanism (like irq number). We currently leave
+		 * this as zero, virtqueues have implicit
+		 * notifications.
+		 *
+		 * The third argument is a "struct hv_ops" containing
+		 * the put_chars() get_chars(), notifier_add() and
+		 * notifier_del() pointers.  The final argument is the
+		 * output buffer size: we can do any size, so we put
+		 * PAGE_SIZE here. */
+		hvc = hvc_alloc(0, 0, &virtio_cons, PAGE_SIZE);
+		if (IS_ERR(hvc)) {
+			ret = PTR_ERR(hvc);
+			goto free_cdev;
+		}
+	}
+
 	pr_info("virtio-serial port found at id %u\n", port_nr);
 
 	return 0;
@@ -721,9 +890,13 @@ static struct virtio_device_id id_table[] = {
 	{ 0 },
 };
 
+static unsigned int features[] = {
+	VIRTIO_CONSOLE_F_SIZE,
+};
+
 static struct virtio_driver virtio_serial = {
-  //	.feature_table = virtserial_features,
-  //	.feature_table_size = ARRAY_SIZE(virtserial_features),
+	.feature_table = features,
+	.feature_table_size = ARRAY_SIZE(features),
 	.driver.name =	KBUILD_MODNAME,
 	.driver.owner =	THIS_MODULE,
 	.id_table =	id_table,
diff --git a/include/linux/virtio_serial.h b/include/linux/virtio_serial.h
index 26ccd83..1c9f853 100644
--- a/include/linux/virtio_serial.h
+++ b/include/linux/virtio_serial.h
@@ -10,7 +10,16 @@
 
 #define VIRTIO_SERIAL_BAD_ID		(~(u32)0)
 
+/* Feature bits */
+#define VIRTIO_CONSOLE_F_SIZE	0	/* Does host provide console size? */
+
 struct virtio_serial_config {
+	/* first two values come from virtio_console */
+	/* colums of the screens */
+	__u16 cols;
+	/* rows of the screens */
+	__u16 rows;
+
 	__u32 max_nr_ports;
 	__u32 nr_active_ports;
 	__u32 ports_map[0 /* (max_nr_ports + 31) / 32 */];
@@ -22,6 +31,9 @@ struct virtio_serial_control {
 	__u16 value;
 };
 
+/* Predefined ports */
+#define VIRTIO_SERIAL_CONSOLE_PORT 2
+
 /* Some events for the control channel */
 /*   Guest -> Host  range 1..256 */
 #define VIRTIO_SERIAL_GUEST_PORT_OPEN	1
@@ -31,7 +43,7 @@ struct virtio_serial_control {
 
 #ifdef __KERNEL__
 
-/* Guest kernel - Guest userspace interface */
+int __init virtio_cons_early_init(int (*put_chars)(u32, const char *, int));
 
 #endif /* __KERNEL__ */
 #endif /* _LINUX_VIRTIO_SERIAL_H */

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-14  8:15                                               ` Amit Shah
@ 2009-08-14 13:29                                                 ` Anthony Liguori
  2009-08-14 13:41                                                   ` Amit Shah
  2009-08-20 13:42                                                   ` Amit Shah
  2009-08-14 13:49                                                 ` Gerd Hoffmann
  1 sibling, 2 replies; 62+ messages in thread
From: Anthony Liguori @ 2009-08-14 13:29 UTC (permalink / raw)
  To: Amit Shah; +Cc: Rusty Russell, virtualization, Gerd Hoffmann, kvm, qemu-devel

Amit Shah wrote:
> On (Mon) Aug 10 2009 [11:59:31], Anthony Liguori wrote:
>   
>> However, as I've mentioned repeatedly, the reason I won't merge  
>> virtio-serial is that it duplicates functionality with virtio-console.   
>> If the two are converged, I'm happy to merge it.  I'm not opposed to  
>> having more functionality.
>>     
>
> The guest code sort-of ends up looking like this after merging
> virtio_console into virtio_serial. Diff is against virtio_serial in my
> git tree, but that should be pretty close to the last submission I made
> at
>
> http://patchwork.kernel.org/patch/39335/
>
> or the tree at
>
> git://git.kernel.org/pub/scm/linux/kernel/git/amit/vs-kernel.git
>
> I've merged bits from virtio_console.c into virtio_serial.c. If needed,
> virtio_serial can be renamed to virtio_console. The VIRITIO_ID also
> needs to change to that of virtio_console's.
>
> Similar changes are needed for userspace.
>
> Since there's support for only one console as of now, I've assigned port
> #2 as the console port so that we hook into hvc when a port is found at
> that location.
>
> One issue that crops up for put_chars: a copy of the buffer to be sent
> has to be made as we don't wait for host to ack the buffer before we
> move on.
>
> The biggest loss so far is Rusty's excellent comments: they will have to
> be reworked and added for the whole of the new file.
>
> Is this approach acceptable?
>   

I think we want to keep virtio_console.c and definitely continue using 
the virtio_console id.  It looks like you are still creating character 
devices as opposed to tty devices.  Is this just an incremental step or 
are you choosing to not do tty devices for a reason (as if so, what's 
that reason)?

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-14 13:29                                                 ` Anthony Liguori
@ 2009-08-14 13:41                                                   ` Amit Shah
  2009-08-20 13:42                                                   ` Amit Shah
  1 sibling, 0 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-14 13:41 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: Rusty Russell, virtualization, Gerd Hoffmann, kvm, qemu-devel

On (Fri) Aug 14 2009 [08:29:28], Anthony Liguori wrote:
> Amit Shah wrote:
>> On (Mon) Aug 10 2009 [11:59:31], Anthony Liguori wrote:
>>   
>>> However, as I've mentioned repeatedly, the reason I won't merge   
>>> virtio-serial is that it duplicates functionality with 
>>> virtio-console.   If the two are converged, I'm happy to merge it.  
>>> I'm not opposed to  having more functionality.
>>>     
>>
>> The guest code sort-of ends up looking like this after merging
>> virtio_console into virtio_serial. Diff is against virtio_serial in my
>> git tree, but that should be pretty close to the last submission I made
>> at
>>
>> http://patchwork.kernel.org/patch/39335/
>>
>> or the tree at
>>
>> git://git.kernel.org/pub/scm/linux/kernel/git/amit/vs-kernel.git
>>
>> I've merged bits from virtio_console.c into virtio_serial.c. If needed,
>> virtio_serial can be renamed to virtio_console. The VIRITIO_ID also
>> needs to change to that of virtio_console's.
>>
>> Similar changes are needed for userspace.
>>
>> Since there's support for only one console as of now, I've assigned port
>> #2 as the console port so that we hook into hvc when a port is found at
>> that location.
>>
>> One issue that crops up for put_chars: a copy of the buffer to be sent
>> has to be made as we don't wait for host to ack the buffer before we
>> move on.
>>
>> The biggest loss so far is Rusty's excellent comments: they will have to
>> be reworked and added for the whole of the new file.
>>
>> Is this approach acceptable?
>>   
>
> I think we want to keep virtio_console.c and definitely continue using  
> the virtio_console id.  It looks like you are still creating character  
> devices as opposed to tty devices.  Is this just an incremental step or  
> are you choosing to not do tty devices for a reason (as if so, what's  
> that reason)?

Just an incremental step. In fact, I haven't yet thought about exposing
a tty device and any problems that might come up. I'll get to doing that
too, of course.

Currently I plan to:
- finish sending connect/disconnect notifications
- finish merge of virtio-console and serial
- look at tty

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-14  8:15                                               ` Amit Shah
  2009-08-14 13:29                                                 ` Anthony Liguori
@ 2009-08-14 13:49                                                 ` Gerd Hoffmann
  2009-08-14 16:25                                                   ` Anthony Liguori
  1 sibling, 1 reply; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-14 13:49 UTC (permalink / raw)
  To: Amit Shah; +Cc: Rusty Russell, virtualization, kvm, qemu-devel

On 08/14/09 10:15, Amit Shah wrote:
> The guest code sort-of ends up looking like this after merging
> virtio_console into virtio_serial.

I think it should better go the other way around: add multichannel 
support to virtio-concole, probably guarded by a feature flag so old 
host+new guest and new host+old guest combinations continue to work.

> Since there's support for only one console as of now, I've assigned port
> #2 as the console port so that we hook into hvc when a port is found at
> that location.

Doesn't sound like this is going to be backward compatible ...

Also I still think passing a 'protocol' string for each port is a good 
idea, so you can stick that into a sysfs file for guests use.

Note it is probably easy to make virtio-console support multiple hvc 
lines by having one protocol for that (named 'console' for example).

cheers,
   Gerd

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-14 13:49                                                 ` Gerd Hoffmann
@ 2009-08-14 16:25                                                   ` Anthony Liguori
  2009-08-20  7:31                                                     ` Rusty Russell
  0 siblings, 1 reply; 62+ messages in thread
From: Anthony Liguori @ 2009-08-14 16:25 UTC (permalink / raw)
  To: Gerd Hoffmann; +Cc: Amit Shah, Rusty Russell, qemu-devel, kvm, virtualization

Gerd Hoffmann wrote:
> On 08/14/09 10:15, Amit Shah wrote:
>> The guest code sort-of ends up looking like this after merging
>> virtio_console into virtio_serial.
>
> I think it should better go the other way around: add multichannel 
> support to virtio-concole, probably guarded by a feature flag so old 
> host+new guest and new host+old guest combinations continue to work.
>
>> Since there's support for only one console as of now, I've assigned port
>> #2 as the console port so that we hook into hvc when a port is found at
>> that location.
>
> Doesn't sound like this is going to be backward compatible ...
>
> Also I still think passing a 'protocol' string for each port is a good 
> idea, so you can stick that into a sysfs file for guests use.

Or drops ports altogether and just use protocol strings...

Regards,

Anthony Liguori

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-14 16:25                                                   ` Anthony Liguori
@ 2009-08-20  7:31                                                     ` Rusty Russell
  2009-08-20  7:44                                                       ` Gerd Hoffmann
  0 siblings, 1 reply; 62+ messages in thread
From: Rusty Russell @ 2009-08-20  7:31 UTC (permalink / raw)
  To: Anthony Liguori; +Cc: Amit Shah, virtualization, Gerd Hoffmann, kvm, qemu-devel

On Sat, 15 Aug 2009 01:55:32 am Anthony Liguori wrote:
> Gerd Hoffmann wrote:
> > Also I still think passing a 'protocol' string for each port is a good 
> > idea, so you can stick that into a sysfs file for guests use.
> 
> Or drops ports altogether and just use protocol strings...

Both is silly, yes.

I guess strings + HAL magic can make the /dev names sane.  I don't want to
see userspace trolling through sysfs to figure out what device to open.

Which is why I prefer assigned numbers, which get mapped to minors.

Rusty.

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for  host-guest communication
  2009-08-20  7:31                                                     ` Rusty Russell
@ 2009-08-20  7:44                                                       ` Gerd Hoffmann
  2009-08-20  7:55                                                         ` Amit Shah
  2009-08-25 12:43                                                         ` Rusty Russell
  0 siblings, 2 replies; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-20  7:44 UTC (permalink / raw)
  To: Rusty Russell; +Cc: Amit Shah, virtualization, kvm, qemu-devel

On 08/20/09 09:31, Rusty Russell wrote:
> On Sat, 15 Aug 2009 01:55:32 am Anthony Liguori wrote:
>> Gerd Hoffmann wrote:
>>> Also I still think passing a 'protocol' string for each port is a good
>>> idea, so you can stick that into a sysfs file for guests use.
>> Or drops ports altogether and just use protocol strings...
>
> Both is silly, yes.
>
> I guess strings + HAL magic can make the /dev names sane.  I don't want to
> see userspace trolling through sysfs to figure out what device to open.

udev can create sane /dev names (or symlinks) by checking sysfs 
attributes, apps just open the /dev/whatever then.

> Which is why I prefer assigned numbers, which get mapped to minors.

ports map trivially to minors.  When using protocol strings minors can 
simply be dynamically auto-allocated by the guest and we don't need the 
port numbers in the host<->guest protocol any more.

I think strings are better as numbers for identifying protocols as you 
can work without a central registry for the numbers then.

cheers,
   Gerd

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for  host-guest communication
  2009-08-20  7:44                                                       ` Gerd Hoffmann
@ 2009-08-20  7:55                                                         ` Amit Shah
  2009-08-20 17:10                                                           ` Jamie Lokier
  2009-08-25 12:43                                                         ` Rusty Russell
  1 sibling, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-20  7:55 UTC (permalink / raw)
  To: Gerd Hoffmann; +Cc: Rusty Russell, qemu-devel, kvm, virtualization

On (Thu) Aug 20 2009 [09:44:29], Gerd Hoffmann wrote:
> On 08/20/09 09:31, Rusty Russell wrote:
>> On Sat, 15 Aug 2009 01:55:32 am Anthony Liguori wrote:
>>> Gerd Hoffmann wrote:
>>>> Also I still think passing a 'protocol' string for each port is a good
>>>> idea, so you can stick that into a sysfs file for guests use.
>>> Or drops ports altogether and just use protocol strings...
>>
>> Both is silly, yes.
>>
>> I guess strings + HAL magic can make the /dev names sane.  I don't want to
>> see userspace trolling through sysfs to figure out what device to open.
>
> udev can create sane /dev names (or symlinks) by checking sysfs  
> attributes, apps just open the /dev/whatever then.

There still will have to be some way in transferring all the strings
from qemu to the guest. Could be done from the config space, but will
have to be done one port at a time (config space is limited in size).

>> Which is why I prefer assigned numbers, which get mapped to minors.
>
> ports map trivially to minors.  When using protocol strings minors can  
> simply be dynamically auto-allocated by the guest and we don't need the  
> port numbers in the host<->guest protocol any more.
>
> I think strings are better as numbers for identifying protocols as you  
> can work without a central registry for the numbers then.

I like the way assigned numbers work: it's simpler to code, needs a
bitmap for all the ports that fits in nicely in the config space and
udev rules / scripts can point /dev/vmch02 to /dev/console.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-14 13:29                                                 ` Anthony Liguori
  2009-08-14 13:41                                                   ` Amit Shah
@ 2009-08-20 13:42                                                   ` Amit Shah
  2009-08-20 14:25                                                     ` Daniel P. Berrange
  1 sibling, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-20 13:42 UTC (permalink / raw)
  To: Anthony Liguori
  Cc: Rusty Russell, virtualization, Gerd Hoffmann, kvm, qemu-devel

On (Fri) Aug 14 2009 [08:29:28], Anthony Liguori wrote:
> Amit Shah wrote:
>> On (Mon) Aug 10 2009 [11:59:31], Anthony Liguori wrote:
>>   
>>> However, as I've mentioned repeatedly, the reason I won't merge   
>>> virtio-serial is that it duplicates functionality with 
>>> virtio-console.   If the two are converged, I'm happy to merge it.  
>>> I'm not opposed to  having more functionality.
>>>     
>>
>> The guest code sort-of ends up looking like this after merging
>> virtio_console into virtio_serial. Diff is against virtio_serial in my
>> git tree, but that should be pretty close to the last submission I made
>> at
>>
>> http://patchwork.kernel.org/patch/39335/
>>
>> or the tree at
>>
>> git://git.kernel.org/pub/scm/linux/kernel/git/amit/vs-kernel.git
>>
>> I've merged bits from virtio_console.c into virtio_serial.c. If needed,
>> virtio_serial can be renamed to virtio_console. The VIRITIO_ID also
>> needs to change to that of virtio_console's.
>>
>> Similar changes are needed for userspace.
>>
>> Since there's support for only one console as of now, I've assigned port
>> #2 as the console port so that we hook into hvc when a port is found at
>> that location.
>>
>> One issue that crops up for put_chars: a copy of the buffer to be sent
>> has to be made as we don't wait for host to ack the buffer before we
>> move on.
>>
>> The biggest loss so far is Rusty's excellent comments: they will have to
>> be reworked and added for the whole of the new file.
>>
>> Is this approach acceptable?
>>   
>
> I think we want to keep virtio_console.c and definitely continue using  
> the virtio_console id.

I've now seen some more code here and to me it looks like virtioconsole
is not used on any of the guests that qemu supports. The virtio_console
kernel module only works with lguest and s390 currently. There is one
feature and some config values supported by the kernel module but not in
qemu.

So it looks as if we have virtio-console merged but no one uses it. Is
this right?

If that's the case, I'll send patches to replace virtio-console with
virtio-serial, making sure we keep the command line arg,

-virtioconsole <qemu char dev>

which will be translated to something like

-virtioserial <qemu char dev>,port=2

or

-virtioserial <qemu char dev>,protocol=console

depending on what we finalise on.

Does anyone have objections to this or pointers for me to see where
virtioconsole is actually used by qemu-supported guests?

I'm also open to convert the guest kernel virtio_console driver over
to support the new functionality, or just let that be and have a new
virtio-serial module.

Rusty, what's your take on this?

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-20 13:42                                                   ` Amit Shah
@ 2009-08-20 14:25                                                     ` Daniel P. Berrange
  2009-08-20 14:38                                                       ` Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Daniel P. Berrange @ 2009-08-20 14:25 UTC (permalink / raw)
  To: Amit Shah; +Cc: kvm, Rusty Russell, qemu-devel, virtualization, Gerd Hoffmann

On Thu, Aug 20, 2009 at 07:12:41PM +0530, Amit Shah wrote:
> 
> I've now seen some more code here and to me it looks like virtioconsole
> is not used on any of the guests that qemu supports. The virtio_console
> kernel module only works with lguest and s390 currently. There is one
> feature and some config values supported by the kernel module but not in
> qemu.
> 
> So it looks as if we have virtio-console merged but no one uses it. Is
> this right?

Nope. Grab a Fedora 11 live CD, and boot with

  # qemu-kvm -virtioconsole stdio -cdrom Fedora-11-i686-Live.iso  -m 500

Once it completes booting & logs into gnome, open a terminal and run
as root

   agetty /dev/hvc0 9600 vt100  

You'll get a login prompt on the host machine now.

What appears to not be working, is early kernel boot messages. eg, I
ought to be able todo

  # qemu-kvm -virtioconsole stdio -kernel vmlinuz -initrd initrd.img \
      -append "console=hvc0"  -m 500

and see the kernel boot messages, but this doesn't work with Fedora
kernels at least. Not tried upstream, or looked to see if this is just
an oversight in the Kconfig use for Fedora kernels.

Regards,
Daniel
-- 
|: Red Hat, Engineering, London   -o-   http://people.redhat.com/berrange/ :|
|: http://libvirt.org  -o-  http://virt-manager.org  -o-  http://ovirt.org :|
|: http://autobuild.org       -o-         http://search.cpan.org/~danberr/ :|
|: GnuPG: 7D3B9505  -o-  F3C9 553F A1DA 4AC2 5648 23C1 B3DF F742 7D3B 9505 :|

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-20 14:25                                                     ` Daniel P. Berrange
@ 2009-08-20 14:38                                                       ` Amit Shah
  2009-08-20 14:42                                                         ` Amit Shah
  0 siblings, 1 reply; 62+ messages in thread
From: Amit Shah @ 2009-08-20 14:38 UTC (permalink / raw)
  To: Daniel P. Berrange
  Cc: Rusty Russell, Gerd Hoffmann, qemu-devel, kvm, virtualization

On (Thu) Aug 20 2009 [15:25:09], Daniel P. Berrange wrote:
> On Thu, Aug 20, 2009 at 07:12:41PM +0530, Amit Shah wrote:
> > 
> > I've now seen some more code here and to me it looks like virtioconsole
> > is not used on any of the guests that qemu supports. The virtio_console
> > kernel module only works with lguest and s390 currently. There is one
> > feature and some config values supported by the kernel module but not in
> > qemu.
> > 
> > So it looks as if we have virtio-console merged but no one uses it. Is
> > this right?
> 
> Nope. Grab a Fedora 11 live CD, and boot with
> 
>   # qemu-kvm -virtioconsole stdio -cdrom Fedora-11-i686-Live.iso  -m 500
> 
> Once it completes booting & logs into gnome, open a terminal and run
> as root
> 
>    agetty /dev/hvc0 9600 vt100  
> 
> You'll get a login prompt on the host machine now.
> 
> What appears to not be working, is early kernel boot messages. eg, I
> ought to be able todo

Oh; ok. So the console device is exposed only in the userspace; it's not
used for the early boot messages and not registered early-on. That's
only done for lguest and s390.

>   # qemu-kvm -virtioconsole stdio -kernel vmlinuz -initrd initrd.img \
>       -append "console=hvc0"  -m 500
> 
> and see the kernel boot messages, but this doesn't work with Fedora
> kernels at least. Not tried upstream, or looked to see if this is just
> an oversight in the Kconfig use for Fedora kernels.

Thanks,

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-20 14:38                                                       ` Amit Shah
@ 2009-08-20 14:42                                                         ` Amit Shah
  0 siblings, 0 replies; 62+ messages in thread
From: Amit Shah @ 2009-08-20 14:42 UTC (permalink / raw)
  To: Daniel P. Berrange
  Cc: Rusty Russell, Gerd Hoffmann, qemu-devel, kvm, virtualization

On (Thu) Aug 20 2009 [20:08:02], Amit Shah wrote:
> On (Thu) Aug 20 2009 [15:25:09], Daniel P. Berrange wrote:
> > On Thu, Aug 20, 2009 at 07:12:41PM +0530, Amit Shah wrote:
> > > 
> > > I've now seen some more code here and to me it looks like virtioconsole
> > > is not used on any of the guests that qemu supports. The virtio_console
> > > kernel module only works with lguest and s390 currently. There is one
> > > feature and some config values supported by the kernel module but not in
> > > qemu.
> > > 
> > > So it looks as if we have virtio-console merged but no one uses it. Is
> > > this right?
> > 
> > Nope. Grab a Fedora 11 live CD, and boot with
> > 
> >   # qemu-kvm -virtioconsole stdio -cdrom Fedora-11-i686-Live.iso  -m 500
> > 
> > Once it completes booting & logs into gnome, open a terminal and run
> > as root
> > 
> >    agetty /dev/hvc0 9600 vt100  
> > 
> > You'll get a login prompt on the host machine now.
> > 
> > What appears to not be working, is early kernel boot messages. eg, I
> > ought to be able todo
> 
> Oh; ok. So the console device is exposed only in the userspace; it's not
> used for the early boot messages and not registered early-on. That's
> only done for lguest and s390.

which, by the way, doesn't change what I wrote above: since it's not
used for earlyboot messages anyway it's not really used as a 'console'
in its real sense and so we could just replace it with the newer version
which makes it much easier for it to work with the kernel driver. Else
supporting older qemu with newer kernel driver will be difficult (if not
impossible).

> >   # qemu-kvm -virtioconsole stdio -kernel vmlinuz -initrd initrd.img \
> >       -append "console=hvc0"  -m 500
> > 
> > and see the kernel boot messages, but this doesn't work with Fedora
> > kernels at least. Not tried upstream, or looked to see if this is just
> > an oversight in the Kconfig use for Fedora kernels.

		Amit

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-20  7:55                                                         ` Amit Shah
@ 2009-08-20 17:10                                                           ` Jamie Lokier
  0 siblings, 0 replies; 62+ messages in thread
From: Jamie Lokier @ 2009-08-20 17:10 UTC (permalink / raw)
  To: Amit Shah; +Cc: Rusty Russell, virtualization, Gerd Hoffmann, kvm, qemu-devel

Amit Shah wrote:
> > I think strings are better as numbers for identifying protocols as you  
> > can work without a central registry for the numbers then.
> 
> I like the way assigned numbers work: it's simpler to code, needs a
> bitmap for all the ports that fits in nicely in the config space and
> udev rules / scripts can point /dev/vmch02 to /dev/console.

How would a third party go about assigning themselves a number?

For the sake of example, imagine they develop a simple service like
"guesttop" which let's the host get a listing of guest processes.

They'll have to distributed app-specific udev rule patches for every
guest distro, which sounds like a lot of work.  The app itself is
probably a very simple C program; the hardest part of making it
portable across distros would be the udev rules, which is silly.

Anyway, every other device has a name or uuid these days.  You can
still use /dev/sda1 to refer to your boot partition, but LABEL=boot is
also available if you prefer.  Isn't that the ethos these days?

Why not both?  /dev/vmch05 if you prefer, plus symlink
/dev/vmch-guesttop -> /dev/vmch05 if name=guesttop was given to QEMU.

If you do stay with numbers only, note that it's not like TCP/UDP port
numbers because the number space is far smaller.  Picking a random
number that you hope nobody else uses is harder.

... Back to technical bits.  If config space is tight, use a channel!
Dedicate channel 0 to control, used to fetch the name (if there is
one) for each number.

-- Jamie

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for host-guest communication
  2009-08-20  7:44                                                       ` Gerd Hoffmann
  2009-08-20  7:55                                                         ` Amit Shah
@ 2009-08-25 12:43                                                         ` Rusty Russell
  2009-08-25 13:00                                                           ` Gerd Hoffmann
  1 sibling, 1 reply; 62+ messages in thread
From: Rusty Russell @ 2009-08-25 12:43 UTC (permalink / raw)
  To: Gerd Hoffmann; +Cc: Amit Shah, virtualization, kvm, qemu-devel

On Thu, 20 Aug 2009 05:14:29 pm Gerd Hoffmann wrote:
> I think strings are better as numbers for identifying protocols as you 
> can work without a central registry for the numbers then.

Yep, all you need is a central registry for names :)

Rusty.

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

* Re: [Qemu-devel] Re: virtio-serial: An interface for  host-guest communication
  2009-08-25 12:43                                                         ` Rusty Russell
@ 2009-08-25 13:00                                                           ` Gerd Hoffmann
  0 siblings, 0 replies; 62+ messages in thread
From: Gerd Hoffmann @ 2009-08-25 13:00 UTC (permalink / raw)
  To: Rusty Russell; +Cc: Amit Shah, virtualization, kvm, qemu-devel

On 08/25/09 14:43, Rusty Russell wrote:
> On Thu, 20 Aug 2009 05:14:29 pm Gerd Hoffmann wrote:
>> I think strings are better as numbers for identifying protocols as you
>> can work without a central registry for the numbers then.
>
> Yep, all you need is a central registry for names :)

There are schemes to do without (reverse domain names, i.e. 
au.com.rustcorp.* is all yours and you don't have to register).  Also 
with names the namespace is much bigger and clashes are much less 
likely, so chances that it works out with everybody simply picking a 
sane name are much higher.

cheers,
   Gerd

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

end of thread, other threads:[~2009-08-25 13:01 UTC | newest]

Thread overview: 62+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2009-07-27 18:04 [Qemu-devel] virtio-serial: An interface for host-guest communication Amit Shah
2009-07-27 18:04 ` [Qemu-devel] [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Amit Shah
2009-07-27 18:04   ` [Qemu-devel] [PATCH 1/3] virtio-serial: virtio device for simple host <-> guest communication Amit Shah
2009-07-27 18:04     ` [Qemu-devel] [PATCH 2/3] vnc: add a is_vnc_active() helper Amit Shah
2009-07-27 18:04       ` [Qemu-devel] [PATCH 3/3] virtio-serial: vnc: support for sending / receiving guest clipboard Amit Shah
2009-08-05  0:03   ` [Qemu-devel] Re: [PATCH 1/1] virtio_serial: A char device for simple guest <-> host communication Rusty Russell
2009-08-05  5:12     ` Amit Shah
2009-08-05  9:58     ` Amit Shah
2009-07-27 20:22 ` [Qemu-devel] Re: virtio-serial: An interface for host-guest communication Anthony Liguori
2009-07-27 20:32   ` Daniel P. Berrange
2009-07-27 20:37     ` Anthony Liguori
2009-07-27 20:46     ` Jamie Lokier
2009-07-27 23:44       ` Anthony Liguori
2009-07-28 10:36         ` Amit Shah
     [not found]           ` <4A6F0048.1000103@codemonkey.ws>
2009-07-29  7:44             ` Amit Shah
2009-07-29  7:48               ` Gleb Natapov
2009-08-05 18:00                 ` Jamie Lokier
     [not found]         ` <20090728140029.GA16067@amd.home.annexia.org>
2009-07-28 14:48           ` Anthony Liguori
2009-07-28 14:55             ` Richard W.M. Jones
2009-07-28 15:00               ` Anthony Liguori
2009-08-03 19:57           ` Anthony Liguori
2009-08-05 17:57             ` Jamie Lokier
2009-08-05 18:00               ` Anthony Liguori
2009-08-06 10:38                 ` Amit Shah
2009-08-06 13:29                   ` Anthony Liguori
2009-08-06 13:41                     ` Amit Shah
2009-08-06 13:58                       ` Anthony Liguori
2009-08-06 14:04                         ` Amit Shah
2009-08-06 17:37                           ` Jamie Lokier
2009-08-07  6:38                             ` Amit Shah
2009-08-07 14:14                               ` Anthony Liguori
2009-08-10  6:55                                 ` Amit Shah
2009-08-10  9:47                                   ` Gerd Hoffmann
2009-08-10 13:02                                     ` Anthony Liguori
2009-08-10 14:02                                       ` Gerd Hoffmann
2009-08-10 14:20                                         ` Anthony Liguori
2009-08-10 15:34                                           ` Gerd Hoffmann
2009-08-10 16:59                                             ` Anthony Liguori
2009-08-10 17:27                                               ` Anthony Liguori
2009-08-12 18:27                                               ` Paul Brook
2009-08-14  8:15                                               ` Amit Shah
2009-08-14 13:29                                                 ` Anthony Liguori
2009-08-14 13:41                                                   ` Amit Shah
2009-08-20 13:42                                                   ` Amit Shah
2009-08-20 14:25                                                     ` Daniel P. Berrange
2009-08-20 14:38                                                       ` Amit Shah
2009-08-20 14:42                                                         ` Amit Shah
2009-08-14 13:49                                                 ` Gerd Hoffmann
2009-08-14 16:25                                                   ` Anthony Liguori
2009-08-20  7:31                                                     ` Rusty Russell
2009-08-20  7:44                                                       ` Gerd Hoffmann
2009-08-20  7:55                                                         ` Amit Shah
2009-08-20 17:10                                                           ` Jamie Lokier
2009-08-25 12:43                                                         ` Rusty Russell
2009-08-25 13:00                                                           ` Gerd Hoffmann
2009-08-10 14:20                                         ` Anthony Liguori
2009-08-10 23:09                                     ` Rusty Russell
2009-08-11  0:16                                       ` Anthony Liguori
2009-08-10 14:27                                   ` Anthony Liguori
2009-08-10 15:57                                     ` Gerd Hoffmann
2009-08-06 10:35               ` Amit Shah
2009-08-05 18:32             ` Richard W.M. Jones

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).