* [Qemu-devel] Multiple port support for virtio-console
@ 2009-09-03 12:53 Amit Shah
0 siblings, 0 replies; 10+ messages in thread
From: Amit Shah @ 2009-09-03 12:53 UTC (permalink / raw)
To: qemu-devel, virtualization, kvm
Hello all,
Here is a new iteration of the patch series that implements a
transport for guest and host communications.
I've tested for compatibility (old qemu & new kernel, new qemu & old
kernel, new qemu & new kernel) and it all works fine*.
There are a few items on my todo list but this works well.
New since last send:
- live migration support**
- write path in the guest driver can now be called from irq context
TODO:
- Convert all config writes to little endian in qemu / convert from
little endian to host endian in guest
- Address a few FIXMEs spread in the code
- Introduce a watermark to stop a rogue host process flooding guest
with data
Conditions:
* Heavy IO on an hvc port and a non-hvc port causes memory corruption
each time, same place. It could be realted to locking but I doubt
that. This still has to be sorted out
** qemu now caches buffers till a char device is opened or till an
entire write request is not received in userspace. These buffers
aren't yet migrated.
Please review.
Amit
^ permalink raw reply [flat|nested] 10+ messages in thread
* [Qemu-devel] Multiple Port Support for virtio-console
@ 2009-09-09 8:11 Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH] virtio_console: Add support for multiple ports for generic guest and host communication Amit Shah
2009-09-10 11:57 ` [Qemu-devel] Re: Multiple Port Support for virtio-console Amit Shah
0 siblings, 2 replies; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:11 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization
Hello all,
Here is a new iteration of the patch series that implements a
transport for guest and host communications.
I've tested for compatibility (old qemu & new kernel, new qemu & old
kernel, new qemu & new kernel) and it all works fine*. Migration works
with the patch Juan just posted.
There are a few items on my todo list but this works well.
New since last send:
- migrate per-port buffers that are queued up
- in-qemu api for open/close/read/write of ports.
- the read() is a callback that's invoked when complete data
corresponding to one write() request on the guest is available.
- removed comments from virtio_console.c that are no longer relevant.
- address review comments by Juan
TODO:
- Convert all config writes to little endian in qemu / convert from
little endian to host endian in guest
- Address a few FIXMEs spread in the code
- Introduce a watermark to stop a rogue host process flooding guest
with data
Conditions:
* Heavy IO on an hvc port and a non-hvc port causes memory corruption
each time, same place. It could be realted to locking but I doubt
that. This still has to be sorted out
Please review.
Amit
^ permalink raw reply [flat|nested] 10+ messages in thread
* [Qemu-devel] [PATCH] virtio_console: Add support for multiple ports for generic guest and host communication
2009-09-09 8:11 [Qemu-devel] Multiple Port Support for virtio-console Amit Shah
@ 2009-09-09 8:12 ` Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 1/5] char: Emit 'OPENED' events on char device open Amit Shah
2009-09-10 11:57 ` [Qemu-devel] Re: Multiple Port Support for virtio-console Amit Shah
1 sibling, 1 reply; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:12 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization; +Cc: Amit Shah
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/vcon2", 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 | 4 +-
drivers/char/virtio_console.c | 1069 +++++++++++++++++++++++++++++++++++-----
include/linux/virtio_console.h | 60 +++-
3 files changed, 994 insertions(+), 139 deletions(-)
diff --git a/drivers/char/Kconfig b/drivers/char/Kconfig
index b8368e2..cbbb9a1 100644
--- a/drivers/char/Kconfig
+++ b/drivers/char/Kconfig
@@ -678,7 +678,9 @@ config VIRTIO_CONSOLE
select HVC_DRIVER
help
Virtio console for use with lguest and other hypervisors.
-
+ Also serves as a general-purpose serial device for data transfer
+ between the guest and host. Character devices at /dev/vconNN will
+ be created when corresponding ports are found.
config HVCS
tristate "IBM Hypervisor Virtual Console Server support"
diff --git a/drivers/char/virtio_console.c b/drivers/char/virtio_console.c
index 0d328b5..3f431bc 100644
--- a/drivers/char/virtio_console.c
+++ b/drivers/char/virtio_console.c
@@ -9,10 +9,8 @@
* functions.
:*/
-/*M:002 The console can be flooded: while the Guest is processing input the
- * Host can send more. Buffering in the Host could alleviate this, but it is a
- * difficult problem in general. :*/
/* Copyright (C) 2006, 2007 Rusty Russell, IBM Corporation
+ * Copyright (C) 2009, Amit Shah, Red Hat, Inc.
*
* 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
@@ -28,116 +26,424 @@
* along with this program; if not, write to the Free Software
* Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
*/
+
+#include <linux/cdev.h>
+#include <linux/device.h>
#include <linux/err.h>
+#include <linux/fs.h>
#include <linux/init.h>
+#include <linux/poll.h>
+#include <linux/spinlock.h>
#include <linux/virtio.h>
#include <linux/virtio_ids.h>
#include <linux/virtio_console.h>
+#include <linux/workqueue.h>
#include "hvc_console.h"
-/*D:340 These represent our input and output console queues, and the virtio
- * operations for them. */
-static struct virtqueue *in_vq, *out_vq;
-static struct virtio_device *vdev;
+/* This struct stores data that's common to all the ports */
+struct virtio_console_struct {
+ /*
+ * Workqueue handlers where we process deferred work after an
+ * interrupt
+ */
+ struct work_struct rx_work;
+ struct work_struct tx_work;
+ struct work_struct config_work;
-/* This is our input buffer, and how much data is left in it. */
-static unsigned int in_len;
-static char *in, *inbuf;
+ struct list_head port_head;
+ struct list_head unused_read_head;
+ struct list_head unused_write_head;
-/* The operations for our console. */
-static struct hv_ops virtio_cons;
+ /* To protect the list of unused write buffers */
+ spinlock_t write_list_lock;
-/* The hvc device */
-static struct hvc_struct *hvc;
+ struct virtio_device *vdev;
+ struct class *class;
+ /* The input and the output queues */
+ struct virtqueue *in_vq, *out_vq;
-/*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)
+ /* The current config space is stored here */
+ struct virtio_console_config *config;
+};
+
+/* This struct holds individual buffers received for each port */
+struct virtio_console_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 holds the per-port data */
+struct virtio_console_port {
+ /* Next port in the list, head is in the virtio_console_struct */
+ struct list_head next;
+
+ /* Buffer management */
+ struct list_head readbuf_head;
+ /* A waitqueue for poll() or blocking read operations */
+ wait_queue_head_t waitqueue;
+
+ /* Each port associates with a separate char device */
+ struct cdev cdev;
+ struct device *dev;
+
+ /* The hvc device, if this port is associated with a console */
+ struct hvc_struct *hvc;
+
+ bool host_connected; /* Is the host device open */
+};
+
+static struct virtio_console_struct virtconsole;
+
+static int major = 60; /* from the experimental range */
+
+static struct virtio_console_port *get_port_from_id(u32 id)
{
- 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();
+ struct virtio_console_port *port;
+
+ list_for_each_entry(port, &virtconsole.port_head, next) {
+ if (MINOR(port->dev->devt) == id)
+ return port;
}
+ return NULL;
+}
- /* We're expected to return the amount of data we wrote: all of it. */
- return count;
+static int get_id_from_port(struct virtio_console_port *port)
+{
+ return MINOR(port->dev->devt);
}
-/* Create a scatter-gather list representing our input buffer and put it in the
- * queue. */
-static void add_inbuf(void)
+static bool is_console_port(struct virtio_console_port *port)
{
- struct scatterlist sg[1];
- sg_init_one(sg, inbuf, PAGE_SIZE);
+ u32 port_nr = get_id_from_port(port);
- /* We should always be able to add one buffer to an empty queue. */
- if (in_vq->vq_ops->add_buf(in_vq, sg, 0, 1, inbuf) < 0)
- BUG();
- in_vq->vq_ops->kick(in_vq);
+ if (port_nr == VIRTIO_CONSOLE_CONSOLE_PORT ||
+ port_nr == VIRTIO_CONSOLE_CONSOLE2_PORT)
+ return true;
+ return false;
}
-/*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)
+static inline bool use_multiport(void)
{
- /* If we don't have an input queue yet, we can't get input. */
- BUG_ON(!in_vq);
+ /*
+ * This condition can be true when put_chars is called from
+ * early_init
+ */
+ if (!virtconsole.vdev)
+ return 0;
+ return virtconsole.vdev->features[0] & (1 << VIRTIO_CONSOLE_F_MULTIPORT);
+}
+
+static inline bool is_internal(u32 flags)
+{
+ return flags & VIRTIO_CONSOLE_ID_INTERNAL;
+}
+
+/*
+ * Give out the data that's requested from the buffers that we have
+ * queued up per port
+ */
+static ssize_t fill_readbuf(struct virtio_console_port *port,
+ char *out_buf, size_t out_count, bool to_user)
+{
+ struct virtio_console_port_buffer *buf, *buf2;
+ ssize_t out_offset, ret;
- /* No buffer? Try to get one. */
- if (!in_len) {
- in = in_vq->vq_ops->get_buf(in_vq, &in_len);
- if (!in)
+ out_offset = 0;
+ list_for_each_entry_safe(buf, buf2, &port->readbuf_head, next) {
+ size_t copy_size;
+
+ copy_size = out_count;
+ if (copy_size > buf->len - buf->offset)
+ copy_size = buf->len - buf->offset;
+
+ 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 */
+ }
+
+ /* Return the number of bytes actually copied */
+ ret = copy_size - ret;
+ buf->offset += ret;
+ out_offset += ret;
+ out_count -= ret;
+
+ if (buf->len - buf->offset == 0) {
+ list_del(&buf->next);
+ kfree(buf->buf);
+ kfree(buf);
+ }
+ if (!out_count)
+ break;
+ }
+ return out_offset;
+}
+
+/* The condition that must be true for polling to end */
+static bool wait_is_over(struct virtio_console_port *port)
+{
+ return !list_empty(&port->readbuf_head) || !port->host_connected;
+}
+
+static ssize_t virtconsole_read(struct file *filp, char __user *ubuf,
+ size_t count, loff_t *offp)
+{
+ struct virtio_console_port *port;
+ ssize_t ret;
+
+ port = filp->private_data;
+
+ if (list_empty(&port->readbuf_head)) {
+ /*
+ * If nothing's connected on the host just return 0 in
+ * case of list_empty; this tells the userspace app
+ * that there's no connection
+ */
+ if (!port->host_connected)
return 0;
+ if (filp->f_flags & O_NONBLOCK)
+ return -EAGAIN;
+
+ ret = wait_event_interruptible(port->waitqueue,
+ wait_is_over(port));
+ if (ret < 0)
+ return ret;
+ }
+ /*
+ * We could've received a disconnection message while we were
+ * waiting for more data.
+ *
+ * This check is not clubbed in the if() statement above as we
+ * might receive some data as well as the host could get
+ * disconnected after we got woken up from our wait. So we
+ * really want to give off whatever data we have and only then
+ * check for host_connected
+ */
+ if (list_empty(&port->readbuf_head) && !port->host_connected)
+ return 0;
+
+ return fill_readbuf(port, ubuf, count, true);
+}
+
+static ssize_t send_buf(struct virtio_console_port *port,
+ const char *in_buf, size_t in_count,
+ u32 flags, bool from_user)
+{
+ struct virtqueue *out_vq;
+ struct virtio_console_port_buffer *buf, *buf2;
+ struct scatterlist sg[1];
+ struct virtio_console_header header;
+ size_t in_offset, copy_size;
+ ssize_t ret;
+ unsigned int header_len;
+
+ if (!in_count)
+ return 0;
+
+ out_vq = virtconsole.out_vq;
+ /*
+ * We should not send internal messages to a host that won't
+ * understand them
+ */
+ if (!use_multiport() && is_internal(flags))
+ return 0;
+ header_len = 0;
+ if (use_multiport()) {
+ header.id = get_id_from_port(port);
+ header.flags = flags;
+ header.size = in_count;
+ header_len = sizeof(header);
+ }
+ in_offset = 0; /* offset in the user buffer */
+ while (in_count - in_offset) {
+ copy_size = min(in_count - in_offset + header_len, PAGE_SIZE);
+
+ spin_lock(&virtconsole.write_list_lock);
+ list_for_each_entry_safe(buf, buf2,
+ &virtconsole.unused_write_head,
+ next) {
+ list_del(&buf->next);
+ break;
+ }
+ spin_unlock(&virtconsole.write_list_lock);
+ if (!buf)
+ break;
+ if (header_len) {
+ memcpy(buf->buf, &header, header_len);
+ copy_size -= header_len;
+ }
+ if (from_user)
+ ret = copy_from_user(buf->buf + header_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 in_buf
+ */
+ memcpy(buf->buf + header_len,
+ in_buf + in_offset, copy_size);
+ ret = 0; /* Emulate copy_from_user behaviour */
+ }
+ buf->len = header_len + copy_size - ret;
+ sg_init_one(sg, buf->buf, buf->len);
+
+ ret = out_vq->vq_ops->add_buf(out_vq, sg, 1, 0, buf);
+ if (ret < 0) {
+ memset(buf->buf, 0, buf->len);
+ spin_lock(&virtconsole.write_list_lock);
+ list_add_tail(&buf->next,
+ &virtconsole.unused_write_head);
+ spin_unlock(&virtconsole.write_list_lock);
+ break;
+ }
+ in_offset += buf->len - header_len;
+ /*
+ * Only send size with the first buffer. This way
+ * userspace can find out a continuous stream of data
+ * belonging to one write request and consume it
+ * appropriately
+ */
+ header.size = 0;
+
+ /* No space left in the vq anyway */
+ if (!ret)
+ break;
}
+ /* Tell Host to go! */
+ out_vq->vq_ops->kick(out_vq);
- /* You want more than we have to give? Well, try wanting less! */
- if (in_len < count)
- count = in_len;
+ /* We're expected to return the amount of data we wrote */
+ return in_offset;
+}
- /* Copy across to their buffer and increment offset. */
- memcpy(buf, in, count);
- in += count;
- in_len -= count;
+static ssize_t virtconsole_write(struct file *filp, const char __user *ubuf,
+ size_t count, loff_t *offp)
+{
+ struct virtio_console_port *port;
- /* Finished? Re-register buffer so Host will use it again. */
- if (in_len == 0)
- add_inbuf();
+ port = filp->private_data;
- return count;
+ return send_buf(port, ubuf, count, 0, true);
}
-/*:*/
-/*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.
+static unsigned int virtconsole_poll(struct file *filp, poll_table *wait)
+{
+ struct virtio_console_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;
+ if (!port->host_connected)
+ ret |= POLLHUP;
+
+ return ret;
+}
+
+static int virtconsole_release(struct inode *inode, struct file *filp)
+{
+ struct virtio_console_control cpkt;
+
+ /* Notify host of port being closed */
+ cpkt.event = VIRTIO_CONSOLE_PORT_OPEN;
+ cpkt.value = 0;
+ send_buf(filp->private_data, (char *)&cpkt, sizeof(cpkt),
+ VIRTIO_CONSOLE_ID_INTERNAL, false);
+ return 0;
+}
+
+static int virtconsole_open(struct inode *inode, struct file *filp)
+{
+ struct cdev *cdev = inode->i_cdev;
+ struct virtio_console_port *port;
+ struct virtio_console_control cpkt;
+
+ port = container_of(cdev, struct virtio_console_port, cdev);
+ filp->private_data = port;
+
+ /* Notify host of port being opened */
+ cpkt.event = VIRTIO_CONSOLE_PORT_OPEN;
+ cpkt.value = 1;
+ send_buf(filp->private_data, (char *)&cpkt, sizeof(cpkt),
+ VIRTIO_CONSOLE_ID_INTERNAL, false);
+
+ return 0;
+}
+
+/*
+ * The file operations that we support: programs in the guest can open
+ * a console device, read from it, write to it, poll for data and
+ * close it. The devices are at /dev/vconNN
+ */
+static const struct file_operations virtconsole_fops = {
+ .owner = THIS_MODULE,
+ .open = virtconsole_open,
+ .read = virtconsole_read,
+ .write = virtconsole_write,
+ .poll = virtconsole_poll,
+ .release = virtconsole_release,
+};
+
+/*D:310
+ * The cons_put_chars() callback is pretty straightforward.
*
- * 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))
+ * We turn the characters into a scatter-gather list, add it to the output
+ * queue and then kick the Host.
+ *
+ * If the data to be outpu spans more than a page, it's split into
+ * page-sized buffers and then individual buffers are pushed to Host.
+ */
+static int cons_put_chars(u32 vtermno, const char *buf, int count)
{
- virtio_cons.put_chars = put_chars;
- return hvc_instantiate(0, 0, &virtio_cons);
+ struct virtio_console_port *port;
+
+ port = get_port_from_id(vtermno);
+ if (!port)
+ return 0;
+
+ return send_buf(port, buf, count, 0, false);
}
+/*D:350
+ * cons_get_chars() is the callback from the hvc_console
+ * infrastructure when an interrupt is received.
+ *
+ * We call out to fill_readbuf that gets us the required data from the
+ * buffers that are queued up.
+ */
+static int cons_get_chars(u32 vtermno, char *buf, int count)
+{
+ struct virtio_console_port *port;
+
+ /* If we don't have an input queue yet, we can't get input. */
+ BUG_ON(!virtconsole.in_vq);
+
+ port = get_port_from_id(vtermno);
+ if (!port)
+ return 0;
+
+ if (list_empty(&port->readbuf_head))
+ return 0;
+
+ return fill_readbuf(port, buf, count, false);
+}
+/*:*/
+
/*
* virtio console configuration. This supports:
* - console resize
@@ -153,98 +459,568 @@ static void virtcons_apply_config(struct virtio_device *dev)
dev->config->get(dev,
offsetof(struct virtio_console_config, rows),
&ws.ws_row, sizeof(u16));
- hvc_resize(hvc, ws);
+ /*
+ * We'll use this way of resizing only for legacy
+ * support. For newer userspace (VIRTIO_CONSOLE_F_MULTPORT+),
+ * use internal messages to indicate console size
+ * changes so that it can be done per-port
+ */
+ hvc_resize(get_port_from_id(VIRTIO_CONSOLE_CONSOLE_PORT)->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)
+static int cons_notifier_add_vio(struct hvc_struct *hp, int data)
{
hp->irq_requested = 1;
- virtcons_apply_config(vdev);
+ virtcons_apply_config(virtconsole.vdev);
return 0;
}
-static void notifier_del_vio(struct hvc_struct *hp, int data)
+static void cons_notifier_del_vio(struct hvc_struct *hp, int data)
{
hp->irq_requested = 0;
}
-static void hvc_handle_input(struct virtqueue *vq)
+/* The operations for our console. */
+static struct hv_ops virtio_cons = {
+ .get_chars = cons_get_chars,
+ .put_chars = cons_put_chars,
+ .notifier_add = cons_notifier_add_vio,
+ .notifier_del = cons_notifier_del_vio,
+ .notifier_hangup = cons_notifier_del_vio,
+};
+
+/*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);
+}
+
+
+/* Any secret messages that the Host and Guest want to share */
+static void handle_control_message(struct virtio_console_port *port,
+ struct virtio_console_control *cpkt)
+{
+ switch (cpkt->event) {
+ case VIRTIO_CONSOLE_PORT_OPEN:
+ port->host_connected = cpkt->value;
+ break;
+ }
+}
+
+
+static struct virtio_console_port_buffer *get_buf(size_t buf_size)
{
- if (hvc_poll(hvc))
+ struct virtio_console_port_buffer *buf;
+
+ buf = kzalloc(sizeof(*buf), GFP_KERNEL);
+ if (!buf)
+ goto out;
+ buf->buf = kzalloc(buf_size, GFP_KERNEL);
+ if (!buf->buf) {
+ kfree(buf);
+ goto out;
+ }
+ buf->len = buf_size;
+out:
+ return buf;
+}
+
+static void fill_queue(struct virtqueue *vq, size_t buf_size,
+ struct list_head *unused_head)
+{
+ struct scatterlist sg[1];
+ struct virtio_console_port_buffer *buf;
+ int ret;
+
+ do {
+ buf = get_buf(buf_size);
+ if (!buf)
+ break;
+ sg_init_one(sg, buf->buf, buf_size);
+
+ ret = vq->vq_ops->add_buf(vq, sg, 0, 1, buf);
+ if (ret < 0) {
+ kfree(buf->buf);
+ kfree(buf);
+ break;
+ }
+ /* We have to keep track of the unused buffers
+ * so that they can be freed when the module
+ * is being removed
+ */
+ list_add_tail(&buf->next, unused_head);
+ } while (ret > 0);
+ vq->vq_ops->kick(vq);
+}
+
+static void fill_receive_queue(void)
+{
+ fill_queue(virtconsole.in_vq, PAGE_SIZE, &virtconsole.unused_read_head);
+}
+
+/*
+ * This function is only called from the init routine so the spinlock
+ * for the unused_write_head list isn't taken
+ */
+static void alloc_write_bufs(void)
+{
+ struct virtio_console_port_buffer *buf;
+ int i;
+
+ for (i = 0; i < 1024; i++) {
+ buf = get_buf(PAGE_SIZE);
+ if (!buf)
+ break;
+ list_add_tail(&buf->next, &virtconsole.unused_write_head);
+ }
+}
+
+/*
+ * The workhandle for any buffers that appear on our input queue.
+ * Pick the buffer; if it's some communication meant for the Guest,
+ * just process it. Otherwise queue it up for the read() or
+ * get_chars() routines to pick the data up later.
+ */
+static void virtio_console_rx_work_handler(struct work_struct *work)
+{
+ struct virtio_console_port *port;
+ struct virtio_console_port_buffer *buf;
+ struct virtio_console_header header;
+ struct virtqueue *vq;
+ unsigned int tmplen, header_len;
+
+ header_len = use_multiport() ? sizeof(header) : 0;
+
+ port = NULL;
+ vq = virtconsole.in_vq;
+ while ((buf = vq->vq_ops->get_buf(vq, &tmplen))) {
+ /* The buffer is no longer unused */
+ list_del(&buf->next);
+
+ if (use_multiport()) {
+ memcpy(&header, buf->buf, header_len);
+ port = get_port_from_id(header.id);
+ } else
+ port = get_port_from_id(VIRTIO_CONSOLE_CONSOLE_PORT);
+ if (!port) {
+ /* No valid header at start of buffer. Drop it. */
+ pr_debug("%s: invalid index in buffer, %c %d\n",
+ __func__, buf->buf[0], buf->buf[0]);
+ /*
+ * OPT: This buffer can be added to the unused
+ * list to avoid free / alloc
+ */
+ kfree(buf->buf);
+ kfree(buf);
+ break;
+ }
+ if (use_multiport() && is_internal(header.flags)) {
+ handle_control_message(port,
+ (struct virtio_console_control *)
+ (buf->buf + header_len));
+ /*
+ * OPT: This buffer can be added to the unused
+ * list to avoid free/alloc
+ */
+ kfree(buf->buf);
+ kfree(buf);
+ } else {
+ buf->len = tmplen;
+ buf->offset = header_len;
+ list_add_tail(&buf->next, &port->readbuf_head);
+ /* We might have missed a connection
+ * notification, e.g. before the queues were
+ * initialised.
+ */
+ port->host_connected = true;
+ }
+ wake_up_interruptible(&port->waitqueue);
+ }
+ if (port && is_console_port(port) && hvc_poll(port->hvc))
hvc_kick();
+
+ /* Allocate buffers for all the ones that got used up */
+ fill_receive_queue();
}
-/*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.
+/*
+ * This is the workhandler for buffers that get received on the output
+ * virtqueue, which is an indication that Host consumed the data we
+ * sent it. Since all our buffers going out are of a fixed size we can
+ * just reuse them instead of freeing them and allocating new ones.
*
- * To set up and manage our virtual console, we call hvc_alloc(). Since we
- * never remove the console device we never need this pointer again.
+ * Zero out the buffer so that we don't leak any information from
+ * other processes. There's a small optimisation here as well: the
+ * buffers are PAGE_SIZE-sized; but instead of zeroing the entire
+ * page, we just zero the length that was most recently used and we
+ * can be sure the rest of the page is already set to 0s.
*
- * Finally we put our input buffer in the input queue, ready to receive. */
-static int __devinit virtcons_probe(struct virtio_device *dev)
+ * So once we zero them out we add them back to the unused buffers
+ * list
+ */
+
+static void virtio_console_tx_work_handler(struct work_struct *work)
{
- vq_callback_t *callbacks[] = { hvc_handle_input, NULL};
+ struct virtqueue *vq;
+ struct virtio_console_port_buffer *buf;
+ unsigned int tmplen;
+
+ vq = virtconsole.out_vq;
+ while ((buf = vq->vq_ops->get_buf(vq, &tmplen))) {
+ /* 0 the buffer to not leak data from other processes */
+ memset(buf->buf, 0, buf->len);
+ spin_lock(&virtconsole.write_list_lock);
+ list_add_tail(&buf->next, &virtconsole.unused_write_head);
+ spin_unlock(&virtconsole.write_list_lock);
+ }
+}
+
+static void rx_intr(struct virtqueue *vq)
+{
+ schedule_work(&virtconsole.rx_work);
+}
+
+static void tx_intr(struct virtqueue *vq)
+{
+ schedule_work(&virtconsole.tx_work);
+}
+
+static void config_intr(struct virtio_device *vdev)
+{
+ /* Handle port hot-add */
+ schedule_work(&virtconsole.config_work);
+
+ /* Handle console size changes */
+ virtcons_apply_config(vdev);
+}
+
+/*
+ * Compare the current config and the new config that we just got and
+ * find out where a particular port was added.
+ */
+static u32 virtconsole_get_hot_add_port(struct virtio_console_config *config)
+{
+ u32 i;
+ u32 port_nr;
+
+ for (i = 0; i < virtconsole.config->max_nr_ports / 32; i++) {
+ port_nr = ffs(config->ports_map[i] ^ virtconsole.config->ports_map[i]);
+ if (port_nr)
+ break;
+ }
+ if (unlikely(!port_nr))
+ return VIRTIO_CONSOLE_BAD_ID;
+
+ /* We used ffs above */
+ port_nr--;
+
+ /* FIXME: Do this only when add_port is successful */
+ virtconsole.config->ports_map[i] |= 1U << port_nr;
+
+ port_nr += i * 32;
+ return port_nr;
+}
+
+/*
+ * Cycle throught the list of active ports and return the next port
+ * that has to be activated.
+ */
+static u32 virtconsole_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 >= virtconsole.config->max_nr_ports / 32))
+ return VIRTIO_CONSOLE_BAD_ID;
+ ++*map_i;
+ *map = virtconsole.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 virtconsole_add_port(u32 port_nr)
+{
+ struct virtio_console_port *port;
+ dev_t devt;
+ int ret;
+
+ port = kzalloc(sizeof(*port), GFP_KERNEL);
+ if (!port)
+ return -ENOMEM;
+
+ devt = MKDEV(major, port_nr);
+ cdev_init(&port->cdev, &virtconsole_fops);
+
+ ret = register_chrdev_region(devt, 1, "virtio-console");
+ 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(virtconsole.class, NULL, devt, NULL,
+ "vcon%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, &virtconsole.port_head);
+
+ if (is_console_port(port)) {
+ /*
+ * To set up and manage our virtual console, we call
+ * hvc_alloc().
+ *
+ * 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.
+ */
+ port->hvc = hvc_alloc(port_nr, 0, &virtio_cons, PAGE_SIZE);
+ if (IS_ERR(port->hvc)) {
+ ret = PTR_ERR(port->hvc);
+ goto free_cdev;
+ }
+ }
+ pr_info("virtio-console port found at id %u\n", port_nr);
+
+ return 0;
+free_cdev:
+ unregister_chrdev(major, "virtio-console");
+ return ret;
+}
+
+/* max_ports is always a multiple of 32; enforced in the Host */
+static u32 get_ports_map_size(u32 max_ports)
+{
+ return sizeof(u32) * (max_ports / 32);
+}
+
+/* The workhandler for config-space updates
+ *
+ * This is used when new ports are added
+ */
+static void virtio_console_config_work_handler(struct work_struct *work)
+{
+ struct virtio_console_config *virtconconf;
+ struct virtio_device *vdev = virtconsole.vdev;
+ u32 i, port_nr;
+ int ret;
+
+ virtconconf = kzalloc(sizeof(*virtconconf) +
+ get_ports_map_size(virtconsole.config->max_nr_ports),
+ GFP_KERNEL);
+ vdev->config->get(vdev,
+ offsetof(struct virtio_console_config, nr_active_ports),
+ &virtconconf->nr_active_ports,
+ sizeof(virtconconf->nr_active_ports));
+ vdev->config->get(vdev,
+ offsetof(struct virtio_console_config, ports_map),
+ virtconconf->ports_map,
+ get_ports_map_size(virtconsole.config->max_nr_ports));
+
+ /* Hot-add ports */
+ for (i = virtconsole.config->nr_active_ports;
+ i < virtconconf->nr_active_ports; i++) {
+ port_nr = virtconsole_get_hot_add_port(virtconconf);
+ if (port_nr == VIRTIO_CONSOLE_BAD_ID)
+ continue;
+ ret = virtconsole_add_port(port_nr);
+ if (!ret)
+ virtconsole.config->nr_active_ports++;
+ }
+ kfree(virtconconf);
+}
+
+/*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.
+ *
+ * Finally we put our input buffer in the input queue, ready to receive.
+ */
+static int __devinit virtcons_probe(struct virtio_device *vdev)
+{
+ vq_callback_t *callbacks[] = { rx_intr, tx_intr };
const char *names[] = { "input", "output" };
struct virtqueue *vqs[2];
- int err;
+ u32 i, map;
+ int ret, map_i;
+ u32 max_nr_ports;
+ bool multiport;
- vdev = dev;
+ virtconsole.vdev = vdev;
- /* This is the scratch page we use to receive console input */
- inbuf = kmalloc(PAGE_SIZE, GFP_KERNEL);
- if (!inbuf) {
- err = -ENOMEM;
- goto fail;
- }
+ multiport = false;
+ if (virtio_has_feature(vdev, VIRTIO_CONSOLE_F_MULTIPORT)) {
+ multiport = true;
+ vdev->features[0] |= 1 << VIRTIO_CONSOLE_F_MULTIPORT;
+ vdev->config->finalize_features(vdev);
+
+ vdev->config->get(vdev,
+ offsetof(struct virtio_console_config,
+ max_nr_ports),
+ &max_nr_ports,
+ sizeof(max_nr_ports));
+ virtconsole.config = kzalloc(sizeof(struct virtio_console_config)
+ + get_ports_map_size(max_nr_ports),
+ GFP_KERNEL);
+ if (!virtconsole.config)
+ return -ENOMEM;
+ virtconsole.config->max_nr_ports = max_nr_ports;
+ vdev->config->get(vdev, offsetof(struct virtio_console_config,
+ nr_active_ports),
+ &virtconsole.config->nr_active_ports,
+ sizeof(virtconsole.config->nr_active_ports));
+ vdev->config->get(vdev,
+ offsetof(struct virtio_console_config,
+ ports_map),
+ virtconsole.config->ports_map,
+ get_ports_map_size(max_nr_ports));
+ }
/* Find the queues. */
/* FIXME: This is why we want to wean off hvc: we do nothing
* when input comes in. */
- err = vdev->config->find_vqs(vdev, 2, vqs, callbacks, names);
- if (err)
- goto free;
+ ret = vdev->config->find_vqs(vdev, 2, vqs, callbacks, names);
+ if (ret)
+ goto fail;
- in_vq = vqs[0];
- out_vq = vqs[1];
+ virtconsole.in_vq = vqs[0];
+ virtconsole.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;
- }
+ INIT_LIST_HEAD(&virtconsole.port_head);
+ INIT_LIST_HEAD(&virtconsole.unused_read_head);
+ INIT_LIST_HEAD(&virtconsole.unused_write_head);
+
+ if (multiport) {
+ map_i = 0;
+ map = virtconsole.config->ports_map[map_i];
+ for (i = 0; i < virtconsole.config->nr_active_ports; i++) {
+ u32 port_nr;
+
+ port_nr = virtconsole_find_next_port(&map, &map_i);
+ if (unlikely(port_nr == VIRTIO_CONSOLE_BAD_ID))
+ continue;
+ virtconsole_add_port(port_nr);
+ }
+ } else
+ virtconsole_add_port(VIRTIO_CONSOLE_CONSOLE_PORT);
+
+ INIT_WORK(&virtconsole.rx_work, &virtio_console_rx_work_handler);
+ INIT_WORK(&virtconsole.tx_work, &virtio_console_tx_work_handler);
+ INIT_WORK(&virtconsole.config_work, &virtio_console_config_work_handler);
+ spin_lock_init(&virtconsole.write_list_lock);
- /* Register the input buffer the first time. */
- add_inbuf();
+ fill_receive_queue();
+ alloc_write_bufs();
return 0;
-free_vqs:
- vdev->config->del_vqs(vdev);
-free:
- kfree(inbuf);
fail:
- return err;
+ return ret;
+}
+
+static void virtcons_remove_port_data(struct virtio_console_port *port)
+{
+ struct virtio_console_port_buffer *buf, *buf2;
+
+#if 0 /* hvc_console is compiled in, at least on Fedora. */
+ if (is_console_port(port))
+ hvc_remove(hvc);
+#endif
+ device_destroy(virtconsole.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 virtcons_remove(struct virtio_device *vdev)
+{
+ struct virtio_console_port *port, *port2;
+ struct virtio_console_port_buffer *buf, *buf2;
+ char *tmpbuf;
+ int len;
+
+ unregister_chrdev(major, "virtio-console");
+ class_destroy(virtconsole.class);
+
+ cancel_work_sync(&virtconsole.rx_work);
+ /*
+ * Free up the buffers that we queued up for the Host to pass
+ * us data
+ */
+ while ((tmpbuf = virtconsole.in_vq->vq_ops->get_buf(virtconsole.in_vq,
+ &len)))
+ kfree(tmpbuf);
+
+ vdev->config->del_vqs(vdev);
+ /*
+ * Free up the buffers that were sent to us by Host but were
+ * left unused
+ */
+ list_for_each_entry_safe(buf, buf2, &virtconsole.unused_read_head, next) {
+ list_del(&buf->next);
+ kfree(buf->buf);
+ kfree(buf);
+ }
+ list_for_each_entry_safe(buf, buf2, &virtconsole.unused_write_head, next) {
+ list_del(&buf->next);
+ kfree(buf->buf);
+ kfree(buf);
+ }
+ list_for_each_entry_safe(port, port2, &virtconsole.port_head, next) {
+ list_del(&port->next);
+ virtcons_remove_port_data(port);
+ kfree(port);
+ }
+ kfree(virtconsole.config);
}
static struct virtio_device_id id_table[] = {
@@ -254,6 +1030,7 @@ static struct virtio_device_id id_table[] = {
static unsigned int features[] = {
VIRTIO_CONSOLE_F_SIZE,
+ VIRTIO_CONSOLE_F_MULTIPORT,
};
static struct virtio_driver virtio_console = {
@@ -263,14 +1040,34 @@ static struct virtio_driver virtio_console = {
.driver.owner = THIS_MODULE,
.id_table = id_table,
.probe = virtcons_probe,
- .config_changed = virtcons_apply_config,
+ .remove = virtcons_remove,
+ .config_changed = config_intr,
};
static int __init init(void)
{
- return register_virtio_driver(&virtio_console);
+ int ret;
+
+ virtconsole.class = class_create(THIS_MODULE, "virtio-console");
+ if (IS_ERR(virtconsole.class)) {
+ pr_err("Error creating virtio-console class\n");
+ ret = PTR_ERR(virtconsole.class);
+ return ret;
+ }
+ ret = register_virtio_driver(&virtio_console);
+ if (ret) {
+ class_destroy(virtconsole.class);
+ return ret;
+ }
+ return 0;
+}
+
+static void __exit fini(void)
+{
+ unregister_virtio_driver(&virtio_console);
}
module_init(init);
+module_exit(fini);
MODULE_DEVICE_TABLE(virtio, id_table);
MODULE_DESCRIPTION("Virtio console driver");
diff --git a/include/linux/virtio_console.h b/include/linux/virtio_console.h
index b5f5198..7f2444c 100644
--- a/include/linux/virtio_console.h
+++ b/include/linux/virtio_console.h
@@ -2,20 +2,76 @@
#define _LINUX_VIRTIO_CONSOLE_H
#include <linux/types.h>
#include <linux/virtio_config.h>
-/* This header, excluding the #ifdef __KERNEL__ part, is BSD licensed so
- * anyone can use the definitions to implement compatible drivers/servers. */
+/*
+ * This header, excluding the #ifdef __KERNEL__ part, is BSD licensed so
+ * anyone can use the definitions to implement compatible drivers/servers.
+ *
+ * Copyright (C) Red Hat, Inc., 2009
+ */
/* Feature bits */
#define VIRTIO_CONSOLE_F_SIZE 0 /* Does host provide console size? */
+#define VIRTIO_CONSOLE_F_MULTIPORT 1 /* Does host provide multiple ports? */
+
+#define VIRTIO_CONSOLE_BAD_ID (~(u32)0) /* Invalid port number */
+
+/* Port at which the virtio console is spawned */
+#define VIRTIO_CONSOLE_CONSOLE_PORT 0
+#define VIRTIO_CONSOLE_CONSOLE2_PORT 1
struct virtio_console_config {
/* colums of the screens */
__u16 cols;
/* rows of the screens */
__u16 rows;
+ /*
+ * max. number of ports supported for each PCI device. Always
+ * a multiple of 32
+ */
+ __u32 max_nr_ports;
+ /* number of ports in use */
+ __u32 nr_active_ports;
+ /*
+ * locations of the ports in use; variable-size array: should
+ * be the last in this struct.
+ */
+ __u32 ports_map[0 /* max_nr_ports / 32 */];
} __attribute__((packed));
+/*
+ * An internal-only message that's passed between the Host and the
+ * Guest for a particular port.
+ */
+struct virtio_console_control {
+ __u16 event;
+ __u16 value;
+};
+
+/* Some events for internal messages (control packets) */
+#define VIRTIO_CONSOLE_PORT_OPEN 0
+
+
+/*
+ * This struct is put in each buffer that gets passed to userspace and
+ * vice-versa
+ */
+struct virtio_console_header {
+ /* Port number */
+ u32 id;
+ /* Some message between host and guest */
+ u32 flags;
+ /*
+ * Complete size of the write request - only sent with the
+ * first buffer for each write request
+ */
+ u32 size;
+} __attribute__((packed));
+
+/* Messages between host and guest ('flags' field in the header above) */
+#define VIRTIO_CONSOLE_ID_INTERNAL (1 << 0)
+
+
#ifdef __KERNEL__
int __init virtio_cons_early_init(int (*put_chars)(u32, const char *, int));
#endif /* __KERNEL__ */
--
1.6.2.5
^ permalink raw reply related [flat|nested] 10+ messages in thread
* [Qemu-devel] [PATCH 1/5] char: Emit 'OPENED' events on char device open
2009-09-09 8:12 ` [Qemu-devel] [PATCH] virtio_console: Add support for multiple ports for generic guest and host communication Amit Shah
@ 2009-09-09 8:12 ` Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 2/5] virtio-console: Add support for multiple ports for generic guest-host communication Amit Shah
0 siblings, 1 reply; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:12 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization; +Cc: Amit Shah
Notify users of the char interface whenever the file / connection is
opened.
The existing RESET event is triggered when the qemu char state is reset
as well; which may not be as interesting as char device open events.
Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
qemu-char.c | 14 +++++++++++++-
qemu-char.h | 1 +
2 files changed, 14 insertions(+), 1 deletions(-)
diff --git a/qemu-char.c b/qemu-char.c
index c25ed1c..5fe882a 100644
--- a/qemu-char.c
+++ b/qemu-char.c
@@ -474,6 +474,8 @@ static CharDriverState *qemu_chr_open_mux(CharDriverState *drv)
chr->chr_write = mux_chr_write;
chr->chr_update_read_handler = mux_chr_update_read_handler;
chr->chr_accept_input = mux_chr_accept_input;
+
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
@@ -622,6 +624,7 @@ static CharDriverState *qemu_chr_open_fd(int fd_in, int fd_out)
chr->chr_close = fd_chr_close;
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
@@ -913,8 +916,10 @@ static void pty_chr_state(CharDriverState *chr, int connected)
* the virtual device linked to our pty. */
qemu_mod_timer(s->timer, qemu_get_clock(rt_clock) + 1000);
} else {
- if (!s->connected)
+ if (!s->connected) {
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
+ }
s->connected = 1;
}
}
@@ -1152,6 +1157,7 @@ static CharDriverState *qemu_chr_open_tty(const char *filename)
}
chr->chr_ioctl = tty_serial_ioctl;
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
#else /* ! __linux__ && ! __sun__ */
@@ -1297,6 +1303,7 @@ static CharDriverState *qemu_chr_open_pp(const char *filename)
chr->opaque = drv;
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
@@ -1576,6 +1583,7 @@ static CharDriverState *qemu_chr_open_win(const char *filename)
return NULL;
}
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
@@ -1675,6 +1683,7 @@ static CharDriverState *qemu_chr_open_win_pipe(const char *filename)
return NULL;
}
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
@@ -1689,6 +1698,7 @@ static CharDriverState *qemu_chr_open_win_file(HANDLE fd_out)
chr->opaque = s;
chr->chr_write = win_chr_write;
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
}
@@ -1823,6 +1833,7 @@ static CharDriverState *qemu_chr_open_udp(const char *def)
chr->chr_write = udp_chr_write;
chr->chr_update_read_handler = udp_chr_update_read_handler;
chr->chr_close = udp_chr_close;
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
return chr;
return_err:
@@ -2029,6 +2040,7 @@ static void tcp_chr_connect(void *opaque)
qemu_set_fd_handler2(s->fd, tcp_chr_read_poll,
tcp_chr_read, NULL, chr);
qemu_chr_reset(chr);
+ qemu_chr_event(chr, CHR_EVENT_OPENED);
}
#define IACSET(x,a,b,c) x[0] = a; x[1] = b; x[2] = c;
diff --git a/qemu-char.h b/qemu-char.h
index df620bc..d725d48 100644
--- a/qemu-char.h
+++ b/qemu-char.h
@@ -12,6 +12,7 @@
#define CHR_EVENT_MUX_IN 3 /* mux-focus was set to this terminal */
#define CHR_EVENT_MUX_OUT 4 /* mux-focus will move on */
#define CHR_EVENT_CLOSED 5 /* connection closed */
+#define CHR_EVENT_OPENED 6 /* connection opened */
#define CHR_IOCTL_SERIAL_SET_PARAMS 1
--
1.6.2.5
^ permalink raw reply related [flat|nested] 10+ messages in thread
* [Qemu-devel] [PATCH 2/5] virtio-console: Add support for multiple ports for generic guest-host communication
2009-09-09 8:12 ` [Qemu-devel] [PATCH 1/5] char: Emit 'OPENED' events on char device open Amit Shah
@ 2009-09-09 8:12 ` Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 3/5] virtio-console: in-qemu api for open/read/write/close ports Amit Shah
0 siblings, 1 reply; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:12 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization; +Cc: Amit Shah
This interface extends the virtio-console device to handle
multiple ports that present 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>
---
hw/pc.c | 16 +-
hw/virtio-console.c | 631 ++++++++++++++++++++++++++++++++++++++++++++++----
hw/virtio-console.h | 53 +++++
monitor.c | 7 +
qemu-monitor.hx | 10 +
qemu-options.hx | 2 +-
sysemu.h | 10 +-
vl.c | 41 ++--
8 files changed, 690 insertions(+), 80 deletions(-)
diff --git a/hw/pc.c b/hw/pc.c
index a84b2d3..46751e6 100644
--- a/hw/pc.c
+++ b/hw/pc.c
@@ -1459,11 +1459,17 @@ static void pc_init1(ram_addr_t ram_size,
}
/* Add virtio console devices */
- if (pci_enabled) {
- for(i = 0; i < MAX_VIRTIO_CONSOLES; i++) {
- if (virtcon_hds[i]) {
- pci_create_simple(pci_bus, -1, "virtio-console-pci");
- }
+ if (pci_enabled && virtcon_nr_ports) {
+ void *dev;
+
+ dev = pci_create_simple(pci_bus, -1, "virtio-console-pci");
+ if (!dev) {
+ fprintf(stderr, "qemu: could not create virtio console pci device\n");
+ exit(1);
+ }
+
+ for (i = 0; i < virtcon_nr_ports; i++) {
+ virtio_console_new_port(dev, virtcon_idx[i]);
}
}
diff --git a/hw/virtio-console.c b/hw/virtio-console.c
index 92c953c..135fdbb 100644
--- a/hw/virtio-console.c
+++ b/hw/virtio-console.c
@@ -2,9 +2,11 @@
* Virtio Console Device
*
* Copyright IBM, Corp. 2008
+ * Copyright Red Hat, Inc. 2009
*
* Authors:
* Christian Ehrhardt <ehrhardt@linux.vnet.ibm.com>
+ * 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.
@@ -12,39 +14,262 @@
*/
#include "hw.h"
+#include "monitor.h"
+#include "pci.h"
+#include "sys-queue.h"
#include "qemu-char.h"
#include "virtio.h"
#include "virtio-console.h"
-
typedef struct VirtIOConsole
{
VirtIODevice vdev;
VirtQueue *ivq, *ovq;
- CharDriverState *chr;
+ struct VirtIOConsolePort *ports;
+ struct virtio_console_config *config;
+ uint32_t guest_features;
} VirtIOConsole;
-static VirtIOConsole *to_virtio_console(VirtIODevice *vdev)
+/* This struct holds individual buffers received for each port */
+typedef struct VirtIOConsolePortBuffer {
+ TAILQ_ENTRY(VirtIOConsolePortBuffer) next;
+
+ uint8_t *buf;
+
+ size_t len; /* length of the buffer */
+
+ /* The size of one write request as issued by the guest. The
+ * buffer could be split in this list but using the size value in
+ * the first buffer for each write we can identify complete
+ * writes
+ */
+ size_t size;
+} VirtIOConsolePortBuffer;
+
+struct VirtIOConsolePort {
+ VirtIOConsole *vcon;
+ CharDriverState *hd;
+
+ TAILQ_HEAD(, VirtIOConsolePortBuffer) unflushed_buffer_head;
+
+ bool guest_connected;
+ bool host_connected;
+};
+
+static VirtIOConsole *virtio_console;
+static struct virtio_console_config virtcon_config;
+static VirtIOConsolePort virtcon_ports[MAX_VIRTIO_CONSOLE_PORTS];
+
+static VirtIOConsolePort *get_port_from_id(uint32_t id)
+{
+ if (id > MAX_VIRTIO_CONSOLE_PORTS)
+ return NULL;
+
+ return &virtcon_ports[id];
+}
+
+static int get_id_from_port(VirtIOConsolePort *port)
+{
+ uint32_t i;
+
+ for (i = 0; i < MAX_VIRTIO_CONSOLE_PORTS; i++) {
+ if (port == &virtcon_ports[i]) {
+ return i;
+ }
+ }
+ return VIRTIO_CONSOLE_BAD_ID;
+}
+
+static bool use_multiport(void)
+{
+ return virtio_console->guest_features & (1 << VIRTIO_CONSOLE_F_MULTIPORT);
+}
+
+static bool is_internal(uint32_t flags)
{
- return (VirtIOConsole *)vdev;
+ return flags & VIRTIO_CONSOLE_ID_INTERNAL;
}
+static bool is_console(uint32_t port_nr)
+{
+ if (port_nr == VIRTIO_CONSOLE_CONSOLE_PORT ||
+ port_nr == VIRTIO_CONSOLE_CONSOLE2_PORT)
+ return true;
+ return false;
+}
+
+void virtio_console_monitor_command(Monitor *mon,
+ const char *command, const char *param)
+{
+ 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_console_port(virtcon_nr_ports, param);
+ if (ret < 0) {
+ monitor_printf(mon, "Error: cannot add new port: %s\n",
+ strerror(-ret));
+ return;
+ }
+ virtio_console_new_port(NULL, virtcon_idx[virtcon_nr_ports]);
+ virtcon_nr_ports++;
+ virtio_console->config->nr_active_ports = cpu_to_le32(virtcon_nr_ports);
+ return;
+ }
+}
+
+static bool has_complete_data(VirtIOConsolePort *port)
+{
+ VirtIOConsolePortBuffer *buf;
+ size_t len, size;
+
+ len = 0;
+ size = 0;
+ TAILQ_FOREACH(buf, &port->unflushed_buffer_head, next) {
+ if (!buf->size && buf == TAILQ_FIRST(&port->unflushed_buffer_head)) {
+ /* We have a buffer that's lost its way; just flush it */
+ return true;
+ }
+ if (size && buf->size) {
+ /* Start of the next write request */
+ return true;
+ }
+ if (buf->size) {
+ size = buf->size;
+ }
+ len += buf->len;
+ if (len == size) {
+ return true;
+ }
+ }
+ return false;
+}
+
+static size_t flush_buf(VirtIOConsolePort *port, const uint8_t *buf, size_t len)
+{
+ if (!port->hd) {
+ return 0;
+ }
+ return qemu_chr_write(port->hd, buf, len);
+}
+
+static void flush_queue(VirtIOConsolePort *port)
+{
+ VirtIOConsolePortBuffer *buf, *buf2;
+ uint8_t *outbuf;
+ size_t outlen;
+
+ while (!TAILQ_EMPTY(&port->unflushed_buffer_head)) {
+ if (!has_complete_data(port)) {
+ break;
+ }
+
+ buf = TAILQ_FIRST(&port->unflushed_buffer_head);
+ if (!buf->size) {
+ /* This is a buf that didn't get consumed as part of a
+ * previous data stream. Bad thing, shouldn't
+ * happen. But let's handle it nonetheless
+ */
+ flush_buf(port, buf->buf, buf->len);
+ TAILQ_REMOVE(&port->unflushed_buffer_head, buf, next);
+ continue;
+ }
+
+ outlen = 0;
+ outbuf = qemu_mallocz(buf->size);
+ TAILQ_FOREACH_SAFE(buf, &port->unflushed_buffer_head, next, buf2) {
+ memcpy(outbuf + outlen, buf->buf, buf->len);
+ outlen += buf->len;
+ TAILQ_REMOVE(&port->unflushed_buffer_head, buf, next);
+ }
+ flush_buf(port, outbuf, outlen);
+ qemu_free(outbuf);
+ }
+}
+
+/* Guest wants to notify us of some event */
+static void handle_control_message(VirtIOConsolePort *port,
+ struct virtio_console_control *cpkt)
+{
+ switch(cpkt->event) {
+ case VIRTIO_CONSOLE_PORT_OPEN:
+ port->guest_connected = cpkt->value;
+ break;
+ }
+}
+
+/* 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_console_handle_output(VirtIODevice *vdev, VirtQueue *vq)
{
- VirtIOConsole *s = to_virtio_console(vdev);
VirtQueueElement elem;
while (virtqueue_pop(vq, &elem)) {
- ssize_t len = 0;
- int d;
+ VirtIOConsolePort *port;
+ VirtIOConsolePortBuffer *buf;
+ struct virtio_console_header header;
+ int header_len;
+
+ buf = qemu_mallocz(sizeof(*buf));
- for (d = 0; d < elem.out_num; d++) {
- len += qemu_chr_write(s->chr, (uint8_t *)elem.out_sg[d].iov_base,
- elem.out_sg[d].iov_len);
+ if (use_multiport()) {
+ header_len = sizeof(header);
+
+ memcpy(&header, elem.out_sg[0].iov_base, header_len);
+ port = get_port_from_id(header.id);
+ if (!port) {
+ qemu_free(buf);
+ goto next_buf;
+ }
+ } else {
+ header_len = 0;
+ port = get_port_from_id(0);
+ }
+
+ /* The guest always sends only one sg */
+ buf->len = elem.out_sg[0].iov_len - header_len;
+ buf->buf = qemu_mallocz(buf->len);
+ memcpy(buf->buf, elem.out_sg[0].iov_base + header_len, buf->len);
+
+ if (use_multiport() && is_internal(header.flags)) {
+ handle_control_message(port,
+ (struct virtio_console_control *)buf->buf);
+ qemu_free(buf->buf);
+ qemu_free(buf);
+ goto next_buf;
+ }
+ if (!use_multiport() || is_console(get_id_from_port(port))) {
+ flush_buf(port, buf->buf, buf->len);
+ qemu_free(buf->buf);
+ qemu_free(buf);
+ goto next_buf;
}
- virtqueue_push(vq, &elem, len);
- virtio_notify(vdev, vq);
+ TAILQ_INSERT_TAIL(&port->unflushed_buffer_head, buf, next);
+ if (use_multiport()) {
+ /* Only the first buffer in a stream will have this
+ * set. This will help us identify the first buffer and
+ * the remaining buffers in the stream based on length
+ */
+ buf->size = header.size;
+ } else {
+ /* We always want to flush all the buffers in this case */
+ buf->size = buf->len;
+ }
+ if (!port->host_connected) {
+ goto next_buf;
+ }
+ flush_queue(port);
+ next_buf:
+ virtqueue_push(vq, &elem, elem.out_sg[0].iov_len);
}
+ virtio_notify(vdev, vq);
}
static void virtio_console_handle_input(VirtIODevice *vdev, VirtQueue *vq)
@@ -53,94 +278,398 @@ static void virtio_console_handle_input(VirtIODevice *vdev, VirtQueue *vq)
static uint32_t virtio_console_get_features(VirtIODevice *vdev)
{
- return 0;
+ return 1 << VIRTIO_CONSOLE_F_MULTIPORT;
}
-static int vcon_can_read(void *opaque)
+static void virtio_console_set_features(VirtIODevice *vdev, uint32_t features)
{
- VirtIOConsole *s = (VirtIOConsole *) opaque;
+ virtio_console->guest_features = features;
+}
- if (!virtio_queue_ready(s->ivq) ||
- !(s->vdev.status & VIRTIO_CONFIG_S_DRIVER_OK) ||
- virtio_queue_empty(s->ivq))
- return 0;
+/* Guest requested config info */
+static void virtio_console_get_config(VirtIODevice *vdev, uint8_t *config_data)
+{
+ memcpy(config_data, &virtcon_config, sizeof(struct virtio_console_config));
+}
- /* current implementations have a page sized buffer.
- * We fall back to a one byte per read if there is not enough room.
- * It would be cool to have a function that returns the available byte
- * instead of checking for a limit */
- if (virtqueue_avail_bytes(s->ivq, TARGET_PAGE_SIZE, 0))
- return TARGET_PAGE_SIZE;
- if (virtqueue_avail_bytes(s->ivq, 1, 0))
- return 1;
- return 0;
+static void virtio_console_set_config(VirtIODevice *vdev,
+ const uint8_t *config_data)
+{
+ struct virtio_console_config config;
+
+ memcpy(&config, config_data, sizeof(config));
}
-static void vcon_read(void *opaque, const uint8_t *buf, int size)
+static size_t write_to_port(VirtIOConsolePort *port,
+ const uint8_t *buf, size_t size, uint32_t flags)
{
- VirtIOConsole *s = (VirtIOConsole *) opaque;
+ VirtQueue *vq = port->vcon->ivq;
VirtQueueElement elem;
- int offset = 0;
+ size_t offset = 0;
+ size_t len = 0;
+
+ if (!virtio_queue_ready(vq)) {
+ return 0;
+ }
+
+ if (!use_multiport() && is_internal(flags)) {
+ return 0;
+ }
- /* The current kernel implementation has only one outstanding input
- * buffer of PAGE_SIZE. Nevertheless, this function is prepared to
- * handle multiple buffers with multiple sg element for input */
while (offset < size) {
- int i = 0;
- if (!virtqueue_pop(s->ivq, &elem))
- break;
- while (offset < size && i < elem.in_num) {
- int len = MIN(elem.in_sg[i].iov_len, size - offset);
- memcpy(elem.in_sg[i].iov_base, buf + offset, len);
+ struct virtio_console_header header;
+ int i, header_len;
+
+ header_len = use_multiport() ? sizeof(header) : 0;
+
+ if (!virtqueue_pop(vq, &elem)) {
+ break;
+ }
+ if (elem.in_sg[0].iov_len < header_len) {
+ /* We can't even store our port number in this buffer. Bug? */
+ fprintf(stderr, "virtio-console: size %zd less than expected\n",
+ elem.in_sg[0].iov_len);
+ exit(1);
+ }
+ header.id = cpu_to_le32(get_id_from_port(port));
+ header.flags = flags;
+ memcpy(elem.in_sg[0].iov_base, &header, header_len);
+
+ for (i = 0; offset < size && i < elem.in_num; i++) {
+ len = MIN(elem.in_sg[i].iov_len - header_len, size - offset);
+
+ memcpy(elem.in_sg[i].iov_base + header_len, buf + offset, len);
offset += len;
- i++;
+ header_len = 0;
}
- virtqueue_push(s->ivq, &elem, size);
+ header_len = use_multiport() ? sizeof(header) : 0;
+ virtqueue_push(vq, &elem, len + header_len);
}
- virtio_notify(&s->vdev, s->ivq);
+ virtio_notify(&port->vcon->vdev, vq);
+ return offset;
+}
+
+static void send_control_event(VirtIOConsolePort *port,
+ struct virtio_console_control *cpkt)
+{
+ write_to_port(port, (uint8_t *)cpkt, sizeof(*cpkt),
+ VIRTIO_CONSOLE_ID_INTERNAL);
+}
+
+/* Readiness of the guest to accept data on a port */
+static int vcon_can_read(void *opaque)
+{
+ VirtIOConsolePort *port = opaque;
+ VirtQueue *vq = port->vcon->ivq;
+ int size, header_len;
+
+ if (use_multiport()) {
+ header_len = sizeof(struct virtio_console_header);
+ } else {
+ header_len = 0;
+ }
+
+ if (!virtio_queue_ready(vq) ||
+ !(port->vcon->vdev.status & VIRTIO_CONFIG_S_DRIVER_OK) ||
+ virtio_queue_empty(vq)) {
+ return 0;
+ }
+ if (!port->guest_connected) {
+ return 0;
+ }
+ size = TARGET_PAGE_SIZE;
+ if (virtqueue_avail_bytes(vq, size, 0)) {
+ return size - header_len;
+ }
+ size = header_len + 1;
+ if (virtqueue_avail_bytes(vq, size, 0)) {
+ return size - header_len;
+ }
+ return 0;
+}
+
+/* Send data from a char device over to the guest */
+static void vcon_read(void *opaque, const uint8_t *buf, int size)
+{
+ VirtIOConsolePort *port = opaque;
+
+ write_to_port(port, buf, size, 0);
}
static void vcon_event(void *opaque, int event)
{
- /* we will ignore any event for the time being */
+ VirtIOConsolePort *port = opaque;
+ struct virtio_console_control cpkt;
+ bool update_needed;
+
+ cpkt.event = VIRTIO_CONSOLE_PORT_OPEN;
+
+ update_needed = false;
+ switch (event) {
+ case CHR_EVENT_OPENED: {
+ cpkt.value = 1;
+ update_needed = true;
+ port->host_connected = true;
+
+ /* Flush any buffers that were pending while the chardev was
+ * disconnected
+ */
+ flush_queue(port);
+ break;
+ }
+ case CHR_EVENT_CLOSED:
+ cpkt.value = 0;
+ update_needed = true;
+ port->host_connected = false;
+ break;
+ default:
+ break;
+ }
+ if (!update_needed) {
+ return;
+ }
+ send_control_event(port, &cpkt);
+}
+
+static void virtio_console_set_port_active(uint32_t idx)
+{
+ int i;
+
+ i = idx / 32;
+ idx %= 32;
+
+ virtcon_config.ports_map[i] |= 1U << idx;
+}
+
+static bool virtio_console_is_port_active(uint32_t idx)
+{
+ int i;
+
+ i = idx / 32;
+ idx %= 32;
+
+ return virtcon_config.ports_map[i] & (1U << idx);
+}
+
+/* This function gets called from vl.c during initial options
+ * parsing as well as from the monitor to parse the options.
+ * So it's a good idea to not print out anything and just
+ * return values which can become meaningful.
+ */
+int init_virtio_console_port(int port, const char *opts)
+{
+ char dev[256];
+ const char *prot;
+ const char *idx;
+ uint32_t port_nr;
+ int j, k;
+
+ memset(dev, 0, sizeof(dev));
+ prot = strstr(opts, ",protocol=");
+ idx = strstr(opts, ",port=");
+
+ port_nr = VIRTIO_CONSOLE_CONSOLE_PORT;
+ if (idx) {
+ port_nr = atol(idx + 6); /* skip ',port=' */
+ }
+ if (port_nr >= MAX_VIRTIO_CONSOLE_PORTS) {
+ return -ENOSPC;
+ }
+ if (virtio_console_is_port_active(port_nr)) {
+ return -EEXIST;
+ }
+
+ /* Just to maintain compatibility with other qemu options,
+ * we have the format of
+ *
+ * -virtioconsole unix:/tmp/foo,protocol=bar,port=3
+ *
+ * so to parse the 'unix:', we have to do the following
+ */
+ j = k = 0;
+ while (opts[j] && &opts[j] != prot && &opts[j] != idx) {
+ dev[k++] = opts[j++];
+ }
+
+ if (dev[0] && strncmp(dev, "none", 4)) {
+ char label[32];
+ snprintf(label, sizeof(label), "virtcon%u", port_nr);
+ virtcon_hds[port] = qemu_chr_open(label, dev, NULL);
+ if (!virtcon_hds[port]) {
+ return -EIO;
+ }
+ }
+ virtcon_idx[port] = port_nr;
+ /* The guest never opens the 'vcon device; it instead uses hvc.
+ * To ensure reads don't block in vcons_can_read on guest_connected,
+ * set it here
+ */
+ if (is_console(port_nr)) {
+ virtcon_ports[port_nr].guest_connected = true;
+ }
+ virtio_console_set_port_active(port_nr);
+
+ return 0;
+}
+
+void *virtio_console_new_port(PCIDevice *dev, uint32_t idx)
+{
+ VirtIOConsolePort *port;
+
+ port = get_port_from_id(idx);
+ port->vcon = virtio_console;
+
+ /* Hot-adding ports to existing device */
+ if (!dev) {
+ dev = port->vcon->vdev.binding_opaque;
+ }
+ port->hd = qdev_init_chardev(&dev->qdev);
+ if (port->hd) {
+ qemu_chr_add_handlers(port->hd, vcon_can_read, vcon_read, vcon_event,
+ port);
+ }
+ TAILQ_INIT(&port->unflushed_buffer_head);
+ /* Send an update to the guest about this new port added */
+ virtio_notify_config(&port->vcon->vdev);
+ return port;
}
static void virtio_console_save(QEMUFile *f, void *opaque)
{
VirtIOConsole *s = opaque;
+ unsigned int i, nr_bufs;
+ /* The virtio device */
virtio_save(&s->vdev, f);
+ /* The config space */
+ qemu_put_be16s(f, &virtcon_config.cols);
+ qemu_put_be16s(f, &virtcon_config.rows);
+ qemu_put_be32s(f, &virtcon_config.max_nr_ports);
+ qemu_put_be32s(f, &virtcon_config.nr_active_ports);
+ for (i = 0; i < le32_to_cpu(virtcon_config.max_nr_ports) / 32; i++) {
+ qemu_put_be32s(f, &virtcon_config.ports_map[i]);
+ }
+ /* Items in struct VirtIOConsole */
+ qemu_put_be32s(f, &s->guest_features);
+ /* Items in struct VirtIOConsolePort */
+ for (i = 0; i < le32_to_cpu(virtcon_config.max_nr_ports); i++) {
+ qemu_put_byte(f, virtcon_ports[i].guest_connected);
+ }
+ /* All the pending buffers from active ports */
+ for (i = 0; i < le32_to_cpu(virtcon_config.max_nr_ports); i++) {
+ VirtIOConsolePortBuffer *buf;
+
+ if (!virtio_console_is_port_active(i)) {
+ continue;
+ }
+ nr_bufs = 0;
+ TAILQ_FOREACH(buf, &virtcon_ports[i].unflushed_buffer_head, next) {
+ nr_bufs++;
+ }
+ /* First the port number, then the nr of bufs and then the bufs */
+ qemu_put_be32s(f, &i);
+ qemu_put_be32s(f, &nr_bufs);
+ if (!nr_bufs) {
+ continue;
+ }
+ TAILQ_FOREACH(buf, &virtcon_ports[i].unflushed_buffer_head, next) {
+ qemu_put_be64s(f, &buf->len);
+ qemu_put_be64s(f, &buf->size);
+ qemu_put_buffer(f, buf->buf, buf->len);
+ }
+ }
}
static int virtio_console_load(QEMUFile *f, void *opaque, int version_id)
{
VirtIOConsole *s = opaque;
+ unsigned int i;
- if (version_id != 1)
+ if (version_id > 2)
return -EINVAL;
+ /* The virtio device */
virtio_load(&s->vdev, f);
+
+ if (version_id < 2)
+ return 0;
+
+ /* The config space */
+ qemu_get_be16s(f, &virtcon_config.cols);
+ qemu_get_be16s(f, &virtcon_config.rows);
+ virtcon_config.max_nr_ports = cpu_to_le32(qemu_get_be32(f));
+ virtcon_config.nr_active_ports = cpu_to_le32(qemu_get_be32(f));
+ for (i = 0; i < le32_to_cpu(virtcon_config.max_nr_ports) / 32; i++) {
+ qemu_get_be32s(f, &virtcon_config.ports_map[i]);
+ }
+ /* Items in struct VirtIOConsole */
+ qemu_get_be32s(f, &virtio_console->guest_features);
+ /* Items in struct VirtIOConsolePort */
+ for (i = 0; i < le32_to_cpu(virtcon_config.max_nr_ports); i++) {
+ virtcon_ports[i].guest_connected = qemu_get_byte(f);
+ }
+
+ /* All the pending buffers from active ports */
+ for (i = 0; i < virtcon_config.nr_active_ports; i++) {
+ VirtIOConsolePortBuffer *buf;
+ unsigned int nr, nr_bufs;
+
+ /* First the port number, then the nr of bufs and then the bufs */
+ qemu_get_be32s(f, &nr);
+ qemu_get_be32s(f, &nr_bufs);
+ if (!nr_bufs) {
+ continue;
+ }
+ for (; nr_bufs; nr_bufs--) {
+ buf = qemu_malloc(sizeof(*buf));
+
+ qemu_get_be64s(f, &buf->len);
+ qemu_get_be64s(f, &buf->size);
+ buf->buf = qemu_malloc(buf->len);
+ qemu_get_buffer(f, buf->buf, buf->len);
+ TAILQ_INSERT_TAIL(&virtcon_ports[nr].unflushed_buffer_head, buf,
+ next);
+ }
+ }
+
return 0;
}
VirtIODevice *virtio_console_init(DeviceState *dev)
{
VirtIOConsole *s;
+
+ if (MAX_VIRTIO_CONSOLE_PORTS % 32) {
+ /* We require MAX_VIRTIO_CONSOLE_PORTS be a multiple of 32:
+ * We anyway use up that much space for the bitmap and it
+ * simplifies some calculations
+ */
+ return NULL;
+ }
+
s = (VirtIOConsole *)virtio_common_init("virtio-console",
VIRTIO_ID_CONSOLE,
- 0, sizeof(VirtIOConsole));
- if (s == NULL)
- return NULL;
+ sizeof(struct virtio_console_config),
+ sizeof(VirtIOConsole));
+ virtio_console = s;
s->vdev.get_features = virtio_console_get_features;
+ s->vdev.set_features = virtio_console_set_features;
+ s->vdev.get_config = virtio_console_get_config;
+ s->vdev.set_config = virtio_console_set_config;
+ /* Add queue for host to guest transfers */
s->ivq = virtio_add_queue(&s->vdev, 128, virtio_console_handle_input);
+ /* Add queue for guest to host transfers */
s->ovq = virtio_add_queue(&s->vdev, 128, virtio_console_handle_output);
- s->chr = qdev_init_chardev(dev);
- qemu_chr_add_handlers(s->chr, vcon_can_read, vcon_read, vcon_event, s);
+ s->ports = virtcon_ports;
+ s->config = &virtcon_config;
+
+ register_savevm("virtio-console", -1, 2, virtio_console_save, virtio_console_load, s);
- register_savevm("virtio-console", -1, 1, virtio_console_save, virtio_console_load, s);
+ virtcon_config.max_nr_ports = cpu_to_le32(MAX_VIRTIO_CONSOLE_PORTS);
+ virtcon_config.nr_active_ports = cpu_to_le32(virtcon_nr_ports);
return &s->vdev;
}
diff --git a/hw/virtio-console.h b/hw/virtio-console.h
index 84d0717..56448a9 100644
--- a/hw/virtio-console.h
+++ b/hw/virtio-console.h
@@ -2,9 +2,11 @@
* Virtio Console Support
*
* Copyright IBM, Corp. 2008
+ * Copyright Red Hat, Inc. 2009
*
* Authors:
* Christian Ehrhardt <ehrhardt@linux.vnet.ibm.com>
+ * 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.
@@ -13,7 +15,58 @@
#ifndef _QEMU_VIRTIO_CONSOLE_H
#define _QEMU_VIRTIO_CONSOLE_H
+#include "sysemu.h"
+
+/* Interface shared between the guest kernel and qemu */
+
/* The ID for virtio console */
#define VIRTIO_ID_CONSOLE 3
+/* Invalid port number */
+#define VIRTIO_CONSOLE_BAD_ID (~(uint32_t)0)
+
+/* Port number to function mapping */
+#define VIRTIO_CONSOLE_CONSOLE_PORT 0
+#define VIRTIO_CONSOLE_CONSOLE2_PORT 1
+
+/* Features supported */
+#define VIRTIO_CONSOLE_F_MULTIPORT 1
+
+struct virtio_console_config
+{
+ /* These two fields are used by VIRTIO_CONSOLE_F_SIZE which
+ * isn't implemented here yet
+ */
+ uint16_t cols;
+ uint16_t rows;
+
+ uint32_t max_nr_ports;
+ uint32_t nr_active_ports;
+ uint32_t ports_map[(MAX_VIRTIO_CONSOLE_PORTS + 31) / 32];
+} __attribute__((packed));
+
+struct virtio_console_control
+{
+ uint16_t event;
+ uint16_t value;
+};
+
+struct virtio_console_header {
+ uint32_t id; /* Port id */
+ uint32_t flags; /* Some message between host and guest */
+ uint32_t size; /* Size that's sent with the first buffer of each stream */
+} __attribute__((packed));
+
+/* Messages between host and guest */
+#define VIRTIO_CONSOLE_ID_INTERNAL (1 << 0)
+
+/* Some events for the internal messages (control packets) */
+#define VIRTIO_CONSOLE_PORT_OPEN 0
+
+
+/* In-qemu interface */
+typedef struct VirtIOConsolePort VirtIOConsolePort;
+void virtio_console_monitor_command(Monitor *mon,
+ const char *command, const char *param);
+
#endif
diff --git a/monitor.c b/monitor.c
index 9839371..5aaeb74 100644
--- a/monitor.c
+++ b/monitor.c
@@ -45,6 +45,7 @@
#include "kvm.h"
#include "acl.h"
#include "exec-all.h"
+#include "hw/virtio-console.h"
#include "qemu-kvm.h"
@@ -1779,6 +1780,12 @@ int monitor_get_fd(Monitor *mon, const char *fdname)
return -1;
}
+static void do_virtio_console_action(Monitor *mon,
+ const char *command, const char *param)
+{
+ virtio_console_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 c193a1e..905cb4c 100644
--- a/qemu-monitor.hx
+++ b/qemu-monitor.hx
@@ -572,6 +572,16 @@ STEXI
Change watchdog action.
ETEXI
+ { "virtio-console", "ss?", do_virtio_console_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-console
+Hot-add ports or send data to virtio-console 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 c1ec976..e2b8e36 100644
--- a/qemu-options.hx
+++ b/qemu-options.hx
@@ -1596,7 +1596,7 @@ ETEXI
DEF("virtioconsole", HAS_ARG, QEMU_OPTION_virtiocon, \
"-virtioconsole c\n" \
- " set virtio console\n")
+ " define virtio console\n")
STEXI
@item -virtioconsole @var{c}
Set virtio console.
diff --git a/sysemu.h b/sysemu.h
index 61c927a..f267058 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"
#include "qemu-option.h"
#include "sys-queue.h"
@@ -233,9 +234,14 @@ extern CharDriverState *parallel_hds[MAX_PARALLEL_PORTS];
/* virtio consoles */
-#define MAX_VIRTIO_CONSOLES 1
+#define MAX_VIRTIO_CONSOLE_PORTS 64 /* Should be a multiple of 32 */
+#define VIRTIO_CONSOLE_PROTO_MAX_LEN 30
-extern CharDriverState *virtcon_hds[MAX_VIRTIO_CONSOLES];
+extern CharDriverState *virtcon_hds[MAX_VIRTIO_CONSOLE_PORTS];
+extern uint32_t virtcon_idx[MAX_VIRTIO_CONSOLE_PORTS];
+extern int virtcon_nr_ports;
+extern int init_virtio_console_port(int port, const char *opts);
+extern void *virtio_console_new_port(PCIDevice *dev, uint32_t idx);
#define TFR(expr) do { if ((expr) != -1) break; } while (errno == EINTR)
diff --git a/vl.c b/vl.c
index 3485ce6..7f7beda 100644
--- a/vl.c
+++ b/vl.c
@@ -211,7 +211,10 @@ static int no_frame = 0;
int no_quit = 0;
CharDriverState *serial_hds[MAX_SERIAL_PORTS];
CharDriverState *parallel_hds[MAX_PARALLEL_PORTS];
-CharDriverState *virtcon_hds[MAX_VIRTIO_CONSOLES];
+CharDriverState *virtcon_hds[MAX_VIRTIO_CONSOLE_PORTS];
+char virtcon_prot[MAX_VIRTIO_CONSOLE_PORTS][VIRTIO_CONSOLE_PROTO_MAX_LEN];
+uint32_t virtcon_idx[MAX_VIRTIO_CONSOLE_PORTS];
+int virtcon_nr_ports;
#ifdef TARGET_I386
int win2k_install_hack = 0;
int rtc_td_hack = 0;
@@ -4922,8 +4925,7 @@ int main(int argc, char **argv, char **envp)
int serial_device_index;
const char *parallel_devices[MAX_PARALLEL_PORTS];
int parallel_device_index;
- const char *virtio_consoles[MAX_VIRTIO_CONSOLES];
- int virtio_console_index;
+ const char *virtio_consoles[MAX_VIRTIO_CONSOLE_PORTS];
const char *loadvm = NULL;
QEMUMachine *machine;
const char *cpu_model;
@@ -4998,9 +5000,9 @@ int main(int argc, char **argv, char **envp)
parallel_devices[i] = NULL;
parallel_device_index = 0;
- for(i = 0; i < MAX_VIRTIO_CONSOLES; i++)
+ for(i = 0; i < MAX_VIRTIO_CONSOLE_PORTS; i++)
virtio_consoles[i] = NULL;
- virtio_console_index = 0;
+ virtcon_nr_ports = 0;
for (i = 0; i < MAX_NODES; i++) {
node_mem[i] = 0;
@@ -5441,12 +5443,12 @@ int main(int argc, char **argv, char **envp)
}
break;
case QEMU_OPTION_virtiocon:
- if (virtio_console_index >= MAX_VIRTIO_CONSOLES) {
+ if (virtcon_nr_ports >= MAX_VIRTIO_CONSOLE_PORTS) {
fprintf(stderr, "qemu: too many virtio consoles\n");
exit(1);
}
- virtio_consoles[virtio_console_index] = optarg;
- virtio_console_index++;
+ virtio_consoles[virtcon_nr_ports] = optarg;
+ virtcon_nr_ports++;
break;
case QEMU_OPTION_parallel:
if (parallel_device_index >= MAX_PARALLEL_PORTS) {
@@ -6033,17 +6035,13 @@ int main(int argc, char **argv, char **envp)
}
}
- for(i = 0; i < MAX_VIRTIO_CONSOLES; i++) {
- const char *devname = virtio_consoles[i];
- if (devname && strcmp(devname, "none")) {
- char label[32];
- snprintf(label, sizeof(label), "virtcon%d", i);
- virtcon_hds[i] = qemu_chr_open(label, devname, NULL);
- if (!virtcon_hds[i]) {
- fprintf(stderr, "qemu: could not open virtio console '%s'\n",
- devname);
- exit(1);
- }
+ for (i = 0; i < virtcon_nr_ports; i++) {
+ int ret;
+
+ ret = init_virtio_console_port(i, virtio_consoles[i]);
+ if (ret < 0) {
+ fprintf(stderr, "qemu: could not init virtio console port at \"%s\"\n", virtio_consoles[i]);
+ exit(1);
}
}
@@ -6164,11 +6162,12 @@ int main(int argc, char **argv, char **envp)
}
}
- for(i = 0; i < MAX_VIRTIO_CONSOLES; i++) {
+ for(i = 0; i < MAX_VIRTIO_CONSOLE_PORTS; i++) {
const char *devname = virtio_consoles[i];
if (virtcon_hds[i] && devname) {
if (strstart(devname, "vc", 0))
- qemu_chr_printf(virtcon_hds[i], "virtio console%d\r\n", i);
+ qemu_chr_printf(virtcon_hds[i], "virtio console%d\r\n",
+ virtcon_idx[i]);
}
}
--
1.6.2.5
^ permalink raw reply related [flat|nested] 10+ messages in thread
* [Qemu-devel] [PATCH 3/5] virtio-console: in-qemu api for open/read/write/close ports
2009-09-09 8:12 ` [Qemu-devel] [PATCH 2/5] virtio-console: Add support for multiple ports for generic guest-host communication Amit Shah
@ 2009-09-09 8:12 ` Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 4/5] vnc: add a is_vnc_active() helper Amit Shah
0 siblings, 1 reply; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:12 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization; +Cc: Amit Shah
This is a simple-to-use api for opening a port, registering
a callback for reading stuff, writing to a port and closing
it.
Another api for hot-adding a port can be provided.
Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
hw/virtio-console.c | 61 ++++++++++++++++++++++++++++++++++++++++++++++++--
hw/virtio-console.h | 5 ++++
2 files changed, 63 insertions(+), 3 deletions(-)
diff --git a/hw/virtio-console.c b/hw/virtio-console.c
index 135fdbb..9e78a9e 100644
--- a/hw/virtio-console.c
+++ b/hw/virtio-console.c
@@ -52,6 +52,11 @@ struct VirtIOConsolePort {
TAILQ_HEAD(, VirtIOConsolePortBuffer) unflushed_buffer_head;
+ /* Callback that's invoked when we have a buffer that can be consumed
+ * by an in-qemu user of this port
+ */
+ size_t (*read_callback)(const uint8_t *buf, const size_t len);
+
bool guest_connected;
bool host_connected;
};
@@ -150,10 +155,15 @@ static bool has_complete_data(VirtIOConsolePort *port)
static size_t flush_buf(VirtIOConsolePort *port, const uint8_t *buf, size_t len)
{
- if (!port->hd) {
- return 0;
+ int ret;
+
+ ret = 0;
+ if (port->read_callback) {
+ ret = port->read_callback(buf, len);
+ } else if (port->hd) {
+ ret = qemu_chr_write(port->hd, buf, len);
}
- return qemu_chr_write(port->hd, buf, len);
+ return ret;
}
static void flush_queue(VirtIOConsolePort *port)
@@ -431,6 +441,51 @@ static void vcon_event(void *opaque, int event)
send_control_event(port, &cpkt);
}
+/* Functions for use inside qemu to open and read from/write to ports */
+VirtIOConsolePort *virtio_console_open(uint32_t id,
+ size_t(*read_callback)(const uint8_t*buf,
+ const size_t len))
+{
+ VirtIOConsolePort *port = get_port_from_id(id);
+ struct virtio_console_control cpkt;
+
+ if (port == NULL) {
+ return NULL;
+ }
+ /* Don't allow opening an already-open port */
+ if (port->host_connected) {
+ return NULL;
+ }
+ /* Send port open notification to the guest */
+ port->host_connected = true;
+ port->read_callback = read_callback;
+ cpkt.event = VIRTIO_CONSOLE_PORT_OPEN;
+ cpkt.value = 1;
+ send_control_event(port, &cpkt);
+ return port;
+}
+
+void virtio_console_close(VirtIOConsolePort *port)
+{
+ struct virtio_console_control cpkt;
+
+ if (!port)
+ return;
+
+ port->read_callback = NULL;
+
+ cpkt.event = VIRTIO_CONSOLE_PORT_OPEN;
+ cpkt.value = 0;
+ send_control_event(port, &cpkt);
+}
+
+size_t virtio_console_write(VirtIOConsolePort *port, uint8_t *buf, size_t size)
+{
+ if (!port || !port->host_connected) {
+ return 0;
+ }
+ return write_to_port(port, buf, size, false);
+}
static void virtio_console_set_port_active(uint32_t idx)
{
int i;
diff --git a/hw/virtio-console.h b/hw/virtio-console.h
index 56448a9..62d0c4b 100644
--- a/hw/virtio-console.h
+++ b/hw/virtio-console.h
@@ -68,5 +68,10 @@ struct virtio_console_header {
typedef struct VirtIOConsolePort VirtIOConsolePort;
void virtio_console_monitor_command(Monitor *mon,
const char *command, const char *param);
+VirtIOConsolePort *virtio_console_open(uint32_t id,
+ size_t(*read_callback)(const uint8_t*buf,
+ const size_t len));
+void virtio_console_close(VirtIOConsolePort *port);
+size_t virtio_console_write(VirtIOConsolePort *port, uint8_t *buf, size_t size);
#endif
--
1.6.2.5
^ permalink raw reply related [flat|nested] 10+ messages in thread
* [Qemu-devel] [PATCH 4/5] vnc: add a is_vnc_active() helper
2009-09-09 8:12 ` [Qemu-devel] [PATCH 3/5] virtio-console: in-qemu api for open/read/write/close ports Amit Shah
@ 2009-09-09 8:12 ` Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 5/5] vnc: Send / receive guest clipboard if virtio-console connected to clipboard port Amit Shah
0 siblings, 1 reply; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:12 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization; +Cc: Amit Shah
This helper is introduced to query the status of vnc.
Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
vnc.c | 10 +++++++++-
1 files changed, 9 insertions(+), 1 deletions(-)
diff --git a/vnc.c b/vnc.c
index 5eaef6a..ff2d4a8 100644
--- a/vnc.c
+++ b/vnc.c
@@ -178,9 +178,17 @@ static void do_info_vnc_client(Monitor *mon, VncState *client)
#endif
}
-void do_info_vnc(Monitor *mon)
+static 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",
--
1.6.2.5
^ permalink raw reply related [flat|nested] 10+ messages in thread
* [Qemu-devel] [PATCH 5/5] vnc: Send / receive guest clipboard if virtio-console connected to clipboard port
2009-09-09 8:12 ` [Qemu-devel] [PATCH 4/5] vnc: add a is_vnc_active() helper Amit Shah
@ 2009-09-09 8:12 ` Amit Shah
0 siblings, 0 replies; 10+ messages in thread
From: Amit Shah @ 2009-09-09 8:12 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization; +Cc: Amit Shah
If a connection to the guest clipboard is open, send the host
clipboard to the guest and guest clipboard to the host on any
change
Signed-off-by: Amit Shah <amit.shah@redhat.com>
---
hw/virtio-console.h | 1 +
vnc.c | 33 +++++++++++++++++++++++++++++++++
2 files changed, 34 insertions(+), 0 deletions(-)
diff --git a/hw/virtio-console.h b/hw/virtio-console.h
index 62d0c4b..bd4c5cb 100644
--- a/hw/virtio-console.h
+++ b/hw/virtio-console.h
@@ -28,6 +28,7 @@
/* Port number to function mapping */
#define VIRTIO_CONSOLE_CONSOLE_PORT 0
#define VIRTIO_CONSOLE_CONSOLE2_PORT 1
+#define VIRTIO_CONSOLE_CLIPBOARD_PORT 3
/* Features supported */
#define VIRTIO_CONSOLE_F_MULTIPORT 1
diff --git a/vnc.c b/vnc.c
index ff2d4a8..19a4cca 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-console.h"
#define VNC_REFRESH_INTERVAL_BASE 30
#define VNC_REFRESH_INTERVAL_INC 50
@@ -47,6 +48,7 @@
static VncDisplay *vnc_display; /* needed for info vnc */
static DisplayChangeListener *dcl;
+static VirtIOConsolePort *virtcon_port;
static char *addr_to_string(const char *format,
struct sockaddr_storage *sa,
@@ -671,6 +673,32 @@ static void vnc_copy(VncState *vs, int src_x, int src_y, int dst_x, int dst_y, i
vnc_flush(vs);
}
+static size_t vnc_clipboard_data_from_guest(const uint8_t *buf, size_t len)
+{
+ VncState *vs;
+ VncDisplay *vd;
+ DisplayState *ds;
+
+ if (!is_vnc_active())
+ return 0;
+
+ 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);
+ }
+ return len;
+}
+
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;
@@ -1240,6 +1268,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_console_write(virtcon_port, text, len);
}
static void check_pointer_type_change(VncState *vs, int absolute)
@@ -2265,6 +2294,7 @@ void vnc_display_close(DisplayState *ds)
vs->subauth = VNC_AUTH_INVALID;
vs->tls.x509verify = 0;
#endif
+ virtio_console_close(virtcon_port);
}
int vnc_display_password(DisplayState *ds, const char *password)
@@ -2514,5 +2544,8 @@ int vnc_display_open(DisplayState *ds, const char *display)
vs->display = dpy;
}
}
+
+ virtcon_port = virtio_console_open(VIRTIO_CONSOLE_CLIPBOARD_PORT,
+ vnc_clipboard_data_from_guest);
return qemu_set_fd_handler2(vs->lsock, NULL, vnc_listen_read, NULL, vs);
}
--
1.6.2.5
^ permalink raw reply related [flat|nested] 10+ messages in thread
* [Qemu-devel] Re: Multiple Port Support for virtio-console
2009-09-09 8:11 [Qemu-devel] Multiple Port Support for virtio-console Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH] virtio_console: Add support for multiple ports for generic guest and host communication Amit Shah
@ 2009-09-10 11:57 ` Amit Shah
1 sibling, 0 replies; 10+ messages in thread
From: Amit Shah @ 2009-09-10 11:57 UTC (permalink / raw)
To: qemu-devel, kvm, virtualization
On (Wed) Sep 09 2009 [13:41:59], Amit Shah wrote:
>
> Hello all,
>
> Here is a new iteration of the patch series that implements a
> transport for guest and host communications.
>
> I've tested for compatibility (old qemu & new kernel, new qemu & old
> kernel, new qemu & new kernel) and it all works fine*. Migration works
> with the patch Juan just posted.
>
> There are a few items on my todo list but this works well.
>
> New since last send:
> - migrate per-port buffers that are queued up
> - in-qemu api for open/close/read/write of ports.
> - the read() is a callback that's invoked when complete data
> corresponding to one write() request on the guest is available.
> - removed comments from virtio_console.c that are no longer relevant.
> - address review comments by Juan
>
> TODO:
> - Convert all config writes to little endian in qemu / convert from
> little endian to host endian in guest
> - Address a few FIXMEs spread in the code
> - Introduce a watermark to stop a rogue host process flooding guest
> with data
>
> Conditions:
> * Heavy IO on an hvc port and a non-hvc port causes memory corruption
> each time, same place. It could be realted to locking but I doubt
> that. This still has to be sorted out
I can't reproduce this badness anymore with the linux-next kernel in the
guest.
So I request the maintainers to please review this patchset for and
consider for inclusion.
Thanks,
Amit
^ permalink raw reply [flat|nested] 10+ messages in thread
* [Qemu-devel] Multiple port support for virtio-console
@ 2009-09-22 16:23 Amit Shah
0 siblings, 0 replies; 10+ messages in thread
From: Amit Shah @ 2009-09-22 16:23 UTC (permalink / raw)
To: qemu-devel
Hello,
This patch series converts virtio-console to qdev, adds a new
virtio-console bus and spawns new ports as devices on the bus.
A few regressions have been introduced (old kernel / new userspace
doesn't work), I'll be looking at that and other issues.
Posting this just to show the approach taken for the new bus.
Amit
^ permalink raw reply [flat|nested] 10+ messages in thread
end of thread, other threads:[~2009-09-22 16:24 UTC | newest]
Thread overview: 10+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2009-09-09 8:11 [Qemu-devel] Multiple Port Support for virtio-console Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH] virtio_console: Add support for multiple ports for generic guest and host communication Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 1/5] char: Emit 'OPENED' events on char device open Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 2/5] virtio-console: Add support for multiple ports for generic guest-host communication Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 3/5] virtio-console: in-qemu api for open/read/write/close ports Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 4/5] vnc: add a is_vnc_active() helper Amit Shah
2009-09-09 8:12 ` [Qemu-devel] [PATCH 5/5] vnc: Send / receive guest clipboard if virtio-console connected to clipboard port Amit Shah
2009-09-10 11:57 ` [Qemu-devel] Re: Multiple Port Support for virtio-console Amit Shah
-- strict thread matches above, loose matches on Subject: below --
2009-09-22 16:23 [Qemu-devel] Multiple port support " Amit Shah
2009-09-03 12:53 Amit Shah
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).