This patch provides initial implementation of read-only subset
of new virtio-fs filesystem. One of the main two parts is new virtio
driver - virtio-fs - that handles interactions with virtio device
as specified in the new 1.2 virtio spec -
https://stefanha.github.io/virtio/virtio-fs.html#x1-41500011. The second
part implements VFS sublayer that provides virtio-fs filesystem in OSv and in 
essence
serves as a passthrough between VFS abstract layer and virtio-fs device and 
host filesystem.

The main motivation is to provide a way to run applications on OSv
without a need to create specific image for each app and not duplicate files
already available on host. In essence one should eventually be able to 
execute an app on Linux host by reading the binaries exposed through
virtio-fs deamon. For that reason the decision to implement the read-inly
subset of the fuse interface was deliberate and we may never implement
full read-write functionality.

Please note that this initial implementation is missing readdir
functionality. Also the implementation of file read operation is
pretty naive and probably quite slow - each read requires an exit to
the host. Eventually we should optimize the latter by using some sort
of cache logic (like in ROFS) and ideally implement DAX (Direct Access) that 
would
provide direct guest/host mapping.

Here are the steps required to mount arbitrary directory from
host and run an app from it:

1. Get latest qemu source from master (> 4.2) and build it along with the 
virtiofsd
daemon the qemu source directory:
   - mkdir build && cd build 
   - ../configure --prefix=$PWD --target-list=x86_64-softmmu
   - make -j8 && make -j4 virtiofsd

2. Apply this patch to OSv tree.

3. Add following line to static/etc/fstab:
   '/dev/virtiofs1 /virtiofs virtiofs defaults 0 0'

4. Add following line to usr.manifest.skel:
   '/virtiofs: ../../static'

5. Build standard OSv image with native-example app and run it with this 
command line to make it set proper command line:
   ./scripts/run.py -e '/virtiofs/hello'

6. Give permissions to virtiofsd daemon to be able to execute unshare 
(http://man7.org/linux/man-pages/man2/unshare.2.html):
   sudo setcap cap_sys_admin+ep build/virtiofsd

7. Start virtiofs daemon in another terminal:
   ./build/virtiofsd --socket-path=/tmp/vhostqemu -o 
source=<OSV_ROOT>/apps/native-example -o cache=always -d

8. Finally run OSv by manually starting qemu with proper parameters (eventually 
we will change run.py to support it). Please see
   parameters for new device:

<QEMU_BUILD_ROOT>/build/x86_64-softmmu/qemu-system-x86_64 \
-m 4G \
-smp 4 \
-vnc :1 \
-gdb tcp::1234,server,nowait \
-device virtio-blk-pci,id=blk0,drive=hd0,scsi=off,bootindex=0 \
-drive file=<OSV_ROOT>/build/last/usr.img,if=none,id=hd0,cache=none,aio=native \
-netdev user,id=un0,net=192.168.122.0/24,host=192.168.122.1 \
-device virtio-net-pci,netdev=un0 \
-device virtio-rng-pci \
-enable-kvm \
-cpu host,+x2apic \
-chardev stdio,mux=on,id=stdio,signal=off \
-mon chardev=stdio,mode=readline \
-device isa-serial,chardev=stdio \
-chardev socket,id=char0,path=/tmp/vhostqemu \
-device vhost-user-fs-pci,queue-size=1024,chardev=char0,tag=myfs \
-object memory-backend-file,id=mem,size=4G,mem-path=/dev/shm,share=on -numa 
node,memdev=mem // This last line may be not necessary

For more info read here - https://virtio-fs.gitlab.io/howto-qemu.html.

For more information information about VirtioFS please read this -
https://vmsplice.net/~stefan/virtio-fs_%20A%20Shared%20File%20System%20for%20Virtual%20Machines.pdf
and https://virtio-fs.gitlab.io/index.html#faq.

Partially implements #1062

Signed-off-by: Waldemar Kozaczuk <jwkozac...@gmail.com>
---
 Makefile                       |   4 +
 arch/x64/arch-setup.cc         |   2 +
 drivers/virtio-fs.cc           | 230 ++++++++++++++++++++++++++
 drivers/virtio-fs.hh           |  64 ++++++++
 drivers/virtio.hh              |   1 +
 fs/vfs/vfs_conf.cc             |   3 +
 fs/virtiofs/virtiofs.hh        |  40 +++++
 fs/virtiofs/virtiofs_i.hh      |  41 +++++
 fs/virtiofs/virtiofs_vfsops.cc | 162 +++++++++++++++++++
 fs/virtiofs/virtiofs_vnops.cc  | 286 +++++++++++++++++++++++++++++++++
 include/osv/device.h           |   1 +
 11 files changed, 834 insertions(+)
 create mode 100644 drivers/virtio-fs.cc
 create mode 100644 drivers/virtio-fs.hh
 create mode 100644 fs/virtiofs/virtiofs.hh
 create mode 100644 fs/virtiofs/virtiofs_i.hh
 create mode 100644 fs/virtiofs/virtiofs_vfsops.cc
 create mode 100644 fs/virtiofs/virtiofs_vnops.cc

diff --git a/Makefile b/Makefile
index 69260dd6..4eef246c 100644
--- a/Makefile
+++ b/Makefile
@@ -829,6 +829,7 @@ drivers += drivers/vmxnet3-queues.o
 drivers += drivers/virtio-blk.o
 drivers += drivers/virtio-scsi.o
 drivers += drivers/virtio-rng.o
+drivers += drivers/virtio-fs.o
 drivers += drivers/kvmclock.o drivers/xenclock.o drivers/hypervclock.o
 drivers += drivers/acpi.o
 drivers += drivers/hpet.o
@@ -1794,6 +1795,9 @@ fs_objs += rofs/rofs_vfsops.o \
        rofs/rofs_cache.o \
        rofs/rofs_common.o
 
+fs_objs += virtiofs/virtiofs_vfsops.o \
+       virtiofs/virtiofs_vnops.o
+
 fs_objs += pseudofs/pseudofs.o
 fs_objs += procfs/procfs_vnops.o
 fs_objs += sysfs/sysfs_vnops.o
diff --git a/arch/x64/arch-setup.cc b/arch/x64/arch-setup.cc
index 986a0928..b29f987c 100644
--- a/arch/x64/arch-setup.cc
+++ b/arch/x64/arch-setup.cc
@@ -256,6 +256,7 @@ void arch_init_premain()
 #include "drivers/virtio-scsi.hh"
 #include "drivers/virtio-net.hh"
 #include "drivers/virtio-rng.hh"
+#include "drivers/virtio-fs.hh"
 #include "drivers/xenplatform-pci.hh"
 #include "drivers/ahci.hh"
 #include "drivers/vmw-pvscsi.hh"
@@ -284,6 +285,7 @@ void arch_init_drivers()
     drvman->register_driver(virtio::scsi::probe);
     drvman->register_driver(virtio::net::probe);
     drvman->register_driver(virtio::rng::probe);
+    drvman->register_driver(virtio::fs::probe);
     drvman->register_driver(xenfront::xenplatform_pci::probe);
     drvman->register_driver(ahci::hba::probe);
     drvman->register_driver(vmw::pvscsi::probe);
diff --git a/drivers/virtio-fs.cc b/drivers/virtio-fs.cc
new file mode 100644
index 00000000..4869306d
--- /dev/null
+++ b/drivers/virtio-fs.cc
@@ -0,0 +1,230 @@
+/*
+ * Copyright (C) 2020 Waldemar Kozaczuk
+ *
+ * This work is open source software, licensed under the terms of the
+ * BSD license as described in the LICENSE file in the top-level directory.
+ */
+
+#include <sys/cdefs.h>
+
+#include "drivers/virtio.hh"
+#include "drivers/virtio-fs.hh"
+#include <osv/interrupt.hh>
+
+#include <osv/mempool.hh>
+#include <osv/mmu.hh>
+
+#include <string>
+#include <string.h>
+#include <map>
+#include <errno.h>
+#include <osv/debug.h>
+
+#include <osv/sched.hh>
+#include "osv/trace.hh"
+#include "osv/aligned_new.hh"
+
+#include <osv/device.h>
+
+using namespace memory;
+
+void fuse_req_wait(struct fuse_request* req)
+{
+    WITH_LOCK(req->req_mutex) {
+        req->req_wait.wait(req->req_mutex);
+    }
+}
+
+namespace virtio {
+
+static int fuse_make_request(void *driver, struct fuse_request* req)
+{
+    auto fs_driver = reinterpret_cast<fs*>(driver);
+    return fs_driver->make_request(req);
+}
+
+static void fuse_req_done(struct fuse_request* req)
+{
+    WITH_LOCK(req->req_mutex) {
+        req->req_wait.wake_one(req->req_mutex);
+    }
+}
+
+static void fuse_req_enqueue_input(vring* queue, struct fuse_request* req)
+{
+    // Header goes first
+    queue->add_out_sg(&req->in_header, sizeof(struct fuse_in_header));
+    //
+    // Add fuse in arguments as out sg
+    if (req->input_args_size) {
+        queue->add_out_sg(req->input_args_data, req->input_args_size);
+    }
+}
+
+static void fuse_req_enqueue_output(vring* queue, struct fuse_request* req)
+{
+    // Header goes first
+    queue->add_in_sg(&req->out_header, sizeof(struct fuse_out_header));
+    //
+    // Add fuse out arguments as in sg
+    if (req->output_args_size) {
+        queue->add_in_sg(req->output_args_data, req->output_args_size);
+    }
+}
+
+int fs::_instance = 0;
+
+static struct devops fs_devops {
+    no_open,
+    no_close,
+    no_read,
+    no_write,
+    no_ioctl,
+    no_devctl,
+    no_strategy,
+};
+
+struct driver fs_driver = {
+    "virtio_fs",
+    &fs_devops,
+    sizeof(struct fuse_strategy),
+};
+
+bool fs::ack_irq()
+{
+    auto isr = _dev.read_and_ack_isr();
+    auto queue = get_virt_queue(VQ_REQUEST);
+
+    if (isr) {
+        queue->disable_interrupts();
+        return true;
+    } else {
+        return false;
+    }
+}
+
+fs::fs(virtio_device& virtio_dev)
+    : virtio_driver(virtio_dev)
+{
+    _driver_name = "virtio-fs";
+    _id = _instance++;
+    virtio_i("VIRTIO FS INSTANCE %d\n", _id);
+
+    // Steps 4, 5 & 6 - negotiate and confirm features
+    setup_features();
+    read_config();
+
+    if (_config.num_queues < 1) {
+        virtio_i("Expected at least one request queue -> baling out!\n");
+        return;
+    }
+
+    // Step 7 - generic init of virtqueues
+    probe_virt_queues();
+
+    //register the single irq callback for the block
+    sched::thread* t = sched::thread::make([this] { this->req_done(); },
+            sched::thread::attr().name("virtio-fs"));
+    t->start();
+    auto queue = get_virt_queue(VQ_REQUEST);
+
+    interrupt_factory int_factory;
+    int_factory.register_msi_bindings = [queue, t](interrupt_manager &msi) {
+        msi.easy_register( {{ VQ_REQUEST, [=] { queue->disable_interrupts(); 
}, t }});
+    };
+
+    int_factory.create_pci_interrupt = [this,t](pci::device &pci_dev) {
+        return new pci_interrupt(
+            pci_dev,
+            [=] { return this->ack_irq(); },
+            [=] { t->wake(); });
+    };
+
+#ifndef AARCH64_PORT_STUB
+    int_factory.create_gsi_edge_interrupt = [this,t]() {
+        return new gsi_edge_interrupt(
+                _dev.get_irq(),
+                [=] { if (this->ack_irq()) t->wake(); });
+    };
+#endif
+
+    _dev.register_interrupt(int_factory);
+
+    // Enable indirect descriptor
+    queue->set_use_indirect(true);
+
+    // Step 8
+    add_dev_status(VIRTIO_CONFIG_S_DRIVER_OK);
+
+    std::string dev_name("virtiofs");
+    dev_name += std::to_string(_disk_idx++);
+
+    struct device *dev = device_create(&fs_driver, dev_name.c_str(), D_BLK); 
//TODO Should it be really D_BLK?
+    struct fuse_strategy *strategy = reinterpret_cast<struct 
fuse_strategy*>(dev->private_data);
+    strategy->drv = this;
+    strategy->make_request = fuse_make_request;
+
+    debugf("virtio-fs: Add device instance %d as [%s]\n", _id, 
dev_name.c_str());
+}
+
+fs::~fs()
+{
+    //TODO: In theory maintain the list of free instances and gc it
+    // including the thread objects and their stack
+}
+
+void fs::read_config()
+{
+    virtio_conf_read(0, &(_config.tag[0]), sizeof(_config.tag));
+    virtio_conf_read(offsetof(fs_config,num_queues), &(_config.num_queues), 
sizeof(_config.num_queues));
+    debugf("virtio-fs: Detected device with tag: [%s] and num_queues: %d\n", 
_config.tag, _config.num_queues);
+}
+
+void fs::req_done()
+{
+    auto* queue = get_virt_queue(VQ_REQUEST);
+    fs_req* req;
+
+    while (1) {
+        virtio_driver::wait_for_queue(queue, &vring::used_ring_not_empty);
+
+        u32 len;
+        while((req = static_cast<fs_req*>(queue->get_buf_elem(&len))) != 
nullptr) {
+            fuse_req_done(req->fuse_req);
+            delete req;
+            queue->get_buf_finalize();
+        }
+
+        // wake up the requesting thread in case the ring was full before
+        queue->wakeup_waiter();
+    }
+}
+
+int fs::make_request(struct fuse_request* req)
+{
+    // The lock is here for parallel requests protection
+    WITH_LOCK(_lock) {
+
+        if (!req) return EIO;
+
+        auto* queue = get_virt_queue(VQ_REQUEST);
+
+        queue->init_sg();
+
+        fuse_req_enqueue_input(queue, req);
+        fuse_req_enqueue_output(queue, req);
+
+        auto* fs_request = new fs_req(req);
+        queue->add_buf_wait(fs_request);
+        queue->kick();
+
+        return 0;
+    }
+}
+
+hw_driver* fs::probe(hw_device* dev)
+{
+    return virtio::probe<fs, VIRTIO_ID_FS>(dev);
+}
+
+}
diff --git a/drivers/virtio-fs.hh b/drivers/virtio-fs.hh
new file mode 100644
index 00000000..efdb956d
--- /dev/null
+++ b/drivers/virtio-fs.hh
@@ -0,0 +1,64 @@
+/*
+ * Copyright (C) 2020 Waldemar Kozaczuk
+ *
+ * This work is open source software, licensed under the terms of the
+ * BSD license as described in the LICENSE file in the top-level directory.
+ */
+
+#ifndef VIRTIO_FS_DRIVER_H
+#define VIRTIO_FS_DRIVER_H
+
+#include <osv/mutex.h>
+#include <osv/waitqueue.hh>
+#include "drivers/virtio.hh"
+#include "drivers/virtio-device.hh"
+#include "fs/virtiofs/virtiofs_i.hh"
+
+namespace virtio {
+
+enum {
+   VQ_HIPRIO,
+   VQ_REQUEST
+};
+
+class fs : public virtio_driver {
+public:
+    struct fs_config {
+        char tag[36];
+        u32 num_queues;
+    } __attribute__((packed));
+
+    explicit fs(virtio_device& dev);
+    virtual ~fs();
+
+    virtual std::string get_name() const { return _driver_name; }
+    void read_config();
+
+    int make_request(struct fuse_request*);
+
+    void req_done();
+    int64_t size();
+
+    bool ack_irq();
+
+    static hw_driver* probe(hw_device* dev);
+private:
+    struct fs_req {
+        fs_req(struct fuse_request* f) :fuse_req(f) {};
+        ~fs_req() {};
+
+        struct fuse_request* fuse_req;
+    };
+
+    std::string _driver_name;
+    fs_config _config;
+
+    //maintains the virtio instance number for multiple drives
+    static int _instance;
+    int _id;
+    // This mutex protects parallel make_request invocations
+    mutex _lock;
+};
+
+}
+#endif
diff --git a/drivers/virtio.hh b/drivers/virtio.hh
index 0cdc6ec5..aa832c51 100644
--- a/drivers/virtio.hh
+++ b/drivers/virtio.hh
@@ -59,6 +59,7 @@ enum {
     VIRTIO_ID_SCSI    = 8,
     VIRTIO_ID_9P      = 9,
     VIRTIO_ID_RPROC_SERIAL = 11,
+    VIRTIO_ID_FS      = 26,
 };
 
 const unsigned max_virtqueues_nr = 64;
diff --git a/fs/vfs/vfs_conf.cc b/fs/vfs/vfs_conf.cc
index ac9c111c..a23ea48f 100644
--- a/fs/vfs/vfs_conf.cc
+++ b/fs/vfs/vfs_conf.cc
@@ -52,9 +52,11 @@ extern struct vfsops nfs_vfsops;
 extern struct vfsops procfs_vfsops;
 extern struct vfsops sysfs_vfsops;
 extern struct vfsops zfs_vfsops;
+extern struct vfsops virtiofs_vfsops;
 
 extern int ramfs_init(void);
 extern int rofs_init(void);
+extern int virtiofs_init(void);
 extern int devfs_init(void);
 extern int nfs_init(void);
 extern int procfs_init(void);
@@ -72,5 +74,6 @@ const struct vfssw vfssw[] = {
     {"sysfs",  sysfs_init,     &sysfs_vfsops},
        {"zfs",         zfs_init,       &zfs_vfsops},
        {"rofs",        rofs_init,      &rofs_vfsops},
+       {"virtiofs",    virtiofs_init,  &virtiofs_vfsops},
        {nullptr,       fs_noop,        nullptr},
 };
diff --git a/fs/virtiofs/virtiofs.hh b/fs/virtiofs/virtiofs.hh
new file mode 100644
index 00000000..892c9ca7
--- /dev/null
+++ b/fs/virtiofs/virtiofs.hh
@@ -0,0 +1,40 @@
+/*
+ * Copyright (C) 2020 Waldemar Kozaczuk
+ *
+ * This work is open source software, licensed under the terms of the
+ * BSD license as described in the LICENSE file in the top-level directory.
+ */
+
+#ifndef __INCLUDE_VIRTIOFS_H__
+#define __INCLUDE_VIRTIOFS_H__
+
+#include <osv/vnode.h>
+#include <osv/mount.h>
+#include <osv/dentry.h>
+#include <osv/prex.h>
+#include <osv/buf.h>
+#include "fuse_kernel.h"
+
+#define VIRTIOFS_DEBUG_ENABLED 1
+
+#if defined(VIRTIOFS_DEBUG_ENABLED)
+#define virtiofs_debug(...) kprintf("[virtiofs] " __VA_ARGS__)
+#else
+#define virtiofs_debug(...)
+#endif
+
+struct virtiofs_inode {
+    uint64_t nodeid;
+    struct fuse_attr attr;
+};
+
+struct virtiofs_file_data {
+    uint64_t file_handle;
+};
+
+void virtiofs_set_vnode(struct vnode *vnode, struct virtiofs_inode *inode);
+
+extern struct vfsops virtiofs_vfsops;
+extern struct vnops virtiofs_vnops;
+
+#endif
diff --git a/fs/virtiofs/virtiofs_i.hh b/fs/virtiofs/virtiofs_i.hh
new file mode 100644
index 00000000..c5dc10d2
--- /dev/null
+++ b/fs/virtiofs/virtiofs_i.hh
@@ -0,0 +1,41 @@
+/*
+ * Copyright (C) 2020 Waldemar Kozaczuk
+ *
+ * This work is open source software, licensed under the terms of the
+ * BSD license as described in the LICENSE file in the top-level directory.
+ */
+
+#ifndef VIRTIOFS_IO_H
+#define VIRTIOFS_IO_H
+
+#include "fuse_kernel.h"
+#include <osv/mutex.h>
+#include <osv/waitqueue.hh>
+
+struct fuse_request
+{
+    struct fuse_in_header in_header;
+    struct fuse_out_header out_header;
+
+    void *input_args_data;
+    size_t input_args_size;
+
+    void *output_args_data;
+    size_t output_args_size;
+
+    mutex_t req_mutex;
+    waitqueue req_wait;
+};
+
+struct fuse_strategy {
+    void *drv;
+    int (*make_request)(void*, struct fuse_request*);
+};
+
+int fuse_req_send_and_receive_reply(fuse_strategy* strategy, uint32_t opcode, 
uint64_t nodeid,
+        void *input_args_data, size_t input_args_size,
+        void *output_args_data, size_t output_args_size);
+
+void fuse_req_wait(struct fuse_request* req);
+
+#endif
diff --git a/fs/virtiofs/virtiofs_vfsops.cc b/fs/virtiofs/virtiofs_vfsops.cc
new file mode 100644
index 00000000..4e8bf26e
--- /dev/null
+++ b/fs/virtiofs/virtiofs_vfsops.cc
@@ -0,0 +1,162 @@
+/*
+ * Copyright (C) 2020 Waldemar Kozaczuk
+ *
+ * This work is open source software, licensed under the terms of the
+ * BSD license as described in the LICENSE file in the top-level directory.
+ */
+
+#include <sys/types.h>
+#include <osv/device.h>
+#include <osv/debug.h>
+#include <iomanip>
+#include <iostream>
+#include "virtiofs.hh"
+#include "virtiofs_i.hh"
+
+static int virtiofs_mount(struct mount *mp, const char *dev, int flags, const 
void *data);
+static int virtiofs_sync(struct mount *mp);
+static int virtiofs_statfs(struct mount *mp, struct statfs *statp);
+static int virtiofs_unmount(struct mount *mp, int flags);
+
+#define virtiofs_vget ((vfsop_vget_t)vfs_nullop)
+
+struct vfsops virtiofs_vfsops = {
+    virtiofs_mount,            /* mount */
+    virtiofs_unmount,  /* unmount */
+    virtiofs_sync,             /* sync */
+    virtiofs_vget,      /* vget */
+    virtiofs_statfs,   /* statfs */
+    &virtiofs_vnops        /* vnops */
+};
+
+std::atomic<uint64_t> fuse_unique_id(1);
+
+int fuse_req_send_and_receive_reply(fuse_strategy* strategy, uint32_t opcode, 
uint64_t nodeid,
+        void *input_args_data, size_t input_args_size, void *output_args_data, 
size_t output_args_size)
+{
+    auto *req = new (std::nothrow) fuse_request();
+
+    req->in_header.len = 0; //TODO
+    req->in_header.opcode = opcode;
+    req->in_header.unique = fuse_unique_id.fetch_add(1, 
std::memory_order_relaxed);
+    req->in_header.nodeid = nodeid;
+    req->in_header.uid = 0;
+    req->in_header.gid = 0;
+    req->in_header.pid = 0;
+
+    req->input_args_data = input_args_data;
+    req->input_args_size = input_args_size;
+
+    req->output_args_data = output_args_data;
+    req->output_args_size = output_args_size;
+
+    assert(strategy->drv);
+    strategy->make_request(strategy->drv, req);
+    fuse_req_wait(req);
+
+    int error = -req->out_header.error;
+    delete req;
+
+    return error;
+}
+
+void virtiofs_set_vnode(struct vnode *vnode, struct virtiofs_inode *inode)
+{
+    if (vnode == nullptr || inode == nullptr) {
+        return;
+    }
+
+    vnode->v_data = inode;
+    vnode->v_ino = inode->nodeid;
+
+    // Set type
+    if (S_ISDIR(inode->attr.mode)) {
+        vnode->v_type = VDIR;
+    } else if (S_ISREG(inode->attr.mode)) {
+        vnode->v_type = VREG;
+    } else if (S_ISLNK(inode->attr.mode)) {
+        vnode->v_type = VLNK;
+    }
+
+    vnode->v_mode = 0555;
+    vnode->v_size = inode->attr.size;
+}
+
+static int
+virtiofs_mount(struct mount *mp, const char *dev, int flags, const void *data) 
{
+    struct device *device;
+    int error = -1;
+
+    error = device_open(dev + 5, DO_RDWR, &device);
+    if (error) {
+        kprintf("[virtiofs] Error opening device!\n");
+        return error;
+    }
+
+    mp->m_dev = device;
+
+    auto *in_args = new(std::nothrow) fuse_init_in();
+    in_args->major = FUSE_KERNEL_VERSION;
+    in_args->minor = FUSE_KERNEL_MINOR_VERSION;
+    in_args->max_readahead = PAGE_SIZE;
+    in_args->flags = 0; //TODO Investigate which flags to set
+
+    auto *out_args = new(std::nothrow) fuse_init_out();
+
+    auto *strategy = reinterpret_cast<fuse_strategy *>(device->private_data);
+    error = fuse_req_send_and_receive_reply(strategy, FUSE_INIT, FUSE_ROOT_ID,
+            in_args, sizeof(*in_args), out_args, sizeof(*out_args));
+
+    if (!error) {
+        virtiofs_debug("Initialized fuse filesystem with version major: %d, 
minor: %d\n",
+                       out_args->major, out_args->minor);
+
+        auto *root_node = new virtiofs_inode();
+        root_node->nodeid = FUSE_ROOT_ID;
+        root_node->attr.mode = S_IFDIR;
+
+        virtiofs_set_vnode(mp->m_root->d_vnode, root_node);
+
+        mp->m_data = strategy;
+        mp->m_dev = device;
+    } else {
+        kprintf("[virtiofs] Failed to initialized fuse filesystem!\n");
+    }
+
+    delete out_args;
+    delete in_args;
+
+    return error;
+}
+
+static int virtiofs_sync(struct mount *mp) {
+    return 0;
+}
+
+static int virtiofs_statfs(struct mount *mp, struct statfs *statp)
+{
+    //TODO
+    //struct virtiofs_info *virtiofs = (struct virtiofs_info *) mp->m_data;
+
+    //statp->f_bsize = sb->block_size;
+
+    // Total blocks
+    //statp->f_blocks = sb->structure_info_blocks_count + 
sb->structure_info_first_block;
+    // Read only. 0 blocks free
+    statp->f_bfree = 0;
+    statp->f_bavail = 0;
+
+    statp->f_ffree = 0;
+    //statp->f_files = sb->inodes_count; //Needs to be inode count
+
+    statp->f_namelen = 0; //FIXME
+
+    return 0;
+}
+
+static int
+virtiofs_unmount(struct mount *mp, int flags)
+{
+    struct device *dev = mp->m_dev;
+    return device_close(dev);
+}
diff --git a/fs/virtiofs/virtiofs_vnops.cc b/fs/virtiofs/virtiofs_vnops.cc
new file mode 100644
index 00000000..3c212274
--- /dev/null
+++ b/fs/virtiofs/virtiofs_vnops.cc
@@ -0,0 +1,286 @@
+/*
+ * Copyright (C) 2020 Waldemar Kozaczuk
+ *
+ * This work is open source software, licensed under the terms of the
+ * BSD license as described in the LICENSE file in the top-level directory.
+ */
+
+#include <sys/stat.h>
+#include <dirent.h>
+#include <sys/param.h>
+
+#include <errno.h>
+#include <string.h>
+#include <stdlib.h>
+#include <fcntl.h>
+
+#include <osv/prex.h>
+#include <osv/vnode.h>
+#include <osv/file.h>
+#include <osv/mount.h>
+#include <osv/debug.h>
+
+#include <sys/types.h>
+#include <osv/device.h>
+#include <osv/sched.hh>
+
+#include "virtiofs.hh"
+#include "virtiofs_i.hh"
+
+#define VERIFY_READ_INPUT_ARGUMENTS() \
+    /* Cant read directories */\
+    if (vnode->v_type == VDIR) \
+        return EISDIR; \
+    /* Cant read anything but reg */\
+    if (vnode->v_type != VREG) \
+        return EINVAL; \
+    /* Cant start reading before the first byte */\
+    if (uio->uio_offset < 0) \
+        return EINVAL; \
+    /* Need to read more than 1 byte */\
+    if (uio->uio_resid == 0) \
+        return 0; \
+    /* Cant read after the end of the file */\
+    if (uio->uio_offset >= (off_t)vnode->v_size) \
+        return 0;
+
+int virtiofs_init(void) {
+    return 0;
+}
+
+static int virtiofs_lookup(struct vnode *vnode, char *name, struct vnode **vpp)
+{
+    struct virtiofs_inode *inode = (struct virtiofs_inode *) vnode->v_data;
+    struct vnode *vp = nullptr;
+
+    if (*name == '\0') {
+        return ENOENT;
+    }
+
+    if (!S_ISDIR(inode->attr.mode)) {
+        kprintf("[virtiofs] inode:%d, ABORTED lookup of %s because not a 
directory\n", inode->nodeid, name);
+        return ENOTDIR;
+    }
+
+    auto *out_args = new (std::nothrow) fuse_entry_out();
+    auto input = new char[strlen(name) + 1];
+    strcpy(input, name);
+
+    auto *strategy = reinterpret_cast<fuse_strategy*>(vnode->v_mount->m_data);
+    int error = fuse_req_send_and_receive_reply(strategy, FUSE_LOOKUP, 
inode->nodeid,
+            input, strlen(name) + 1, out_args, sizeof(*out_args));
+
+    if (!error) {
+        if (vget(vnode->v_mount, out_args->nodeid, &vp)) { //TODO: Will it 
ever work? Revisit
+            virtiofs_debug("lookup found vp in cache!\n");
+            *vpp = vp;
+            return 0;
+        }
+
+        auto *new_inode = new virtiofs_inode();
+        new_inode->nodeid = out_args->nodeid;
+        virtiofs_debug("inode %d, lookup found inode %d for %s!\n", 
inode->nodeid, new_inode->nodeid, name);
+        memcpy(&new_inode->attr, &out_args->attr, sizeof(out_args->attr));
+
+        virtiofs_set_vnode(vp, new_inode);
+        *vpp = vp;
+    } else {
+        kprintf("[virtiofs] inode:%d, lookup failed to find %s\n", 
inode->nodeid, name);
+        //TODO Implement proper error handling by sending FUSE_FORGET
+    }
+
+    delete input;
+    delete out_args;
+
+    return error;
+}
+
+static int virtiofs_open(struct file *fp)
+{
+    if ((file_flags(fp) & FWRITE)) {
+        // Do no allow opening files to write
+        return (EROFS);
+    }
+
+    struct vnode *vnode = file_dentry(fp)->d_vnode;
+    struct virtiofs_inode *inode = (struct virtiofs_inode *) vnode->v_data;
+
+    auto *out_args = new (std::nothrow) fuse_open_out();
+    auto *input_args = new (std::nothrow) fuse_open_in();
+    input_args->flags = O_RDONLY;
+
+    auto *strategy = reinterpret_cast<fuse_strategy*>(vnode->v_mount->m_data);
+    int error = fuse_req_send_and_receive_reply(strategy, FUSE_OPEN, 
inode->nodeid,
+            input_args, sizeof(*input_args), out_args, sizeof(*out_args));
+
+    if (!error) {
+        virtiofs_debug("inode %d, opened\n", inode->nodeid);
+
+        auto *file_data = new virtiofs_file_data();
+        file_data->file_handle = out_args->fh;
+        fp->f_data = file_data;
+    }
+
+    delete input_args;
+    delete out_args;
+
+    return error;
+}
+
+static int virtiofs_close(struct vnode *vnode, struct file *fp)
+{
+    struct virtiofs_inode *inode = (struct virtiofs_inode *) vnode->v_data;
+
+    auto *input_args = new (std::nothrow) fuse_release_in();
+    auto *file_data = reinterpret_cast<virtiofs_file_data*>(fp->f_data);
+    input_args->fh = file_data->file_handle;
+
+    auto *strategy = reinterpret_cast<fuse_strategy*>(vnode->v_mount->m_data);
+    auto error = fuse_req_send_and_receive_reply(strategy, FUSE_RELEASE, 
inode->nodeid,
+            input_args, sizeof(*input_args), nullptr, 0);
+
+    if (!error) {
+        fp->f_data = nullptr;
+        delete file_data;
+        virtiofs_debug("inode %d, closed\n", inode->nodeid);
+    }
+
+    //TODO: Investigate if we should send FUSE_FORGET once all handles to the 
file closed on our side
+
+    delete input_args;
+
+    return error;
+}
+
+static int virtiofs_readlink(struct vnode *vnode, struct uio *uio)
+{
+    struct virtiofs_inode *inode = (struct virtiofs_inode *) vnode->v_data;
+
+    auto *link_path = new (std::nothrow) char[PATH_MAX];
+
+    auto *strategy = reinterpret_cast<fuse_strategy*>(vnode->v_mount->m_data);
+    int error = fuse_req_send_and_receive_reply(strategy, FUSE_READLINK, 
inode->nodeid,
+            nullptr, 0, link_path, PATH_MAX);
+
+    int ret = 0;
+    if (!error) {
+        virtiofs_debug("inode %d, read symlink [%s]\n", inode->nodeid, 
link_path);
+        ret = uiomove(link_path, strlen(link_path), uio);
+    } else {
+        kprintf("[virtiofs] Error reading data\n");
+        ret = error;
+    }
+
+    delete link_path;
+
+    return ret;
+}
+
+//TODO: Optimize it to reduce number of exits to host (each 
fuse_req_send_and_receive_reply())
+// by reading eagerly "ahead/around" just like ROFS does and caching it
+static int virtiofs_read(struct vnode *vnode, struct file *fp, struct uio 
*uio, int ioflag)
+{
+    struct virtiofs_inode *inode = (struct virtiofs_inode *) vnode->v_data;
+
+    VERIFY_READ_INPUT_ARGUMENTS()
+
+    // Total read amount is what they requested, or what is left
+    uint64_t read_amt = std::min<uint64_t>(inode->attr.size - uio->uio_offset, 
uio->uio_resid);
+    void *buf = malloc(read_amt);
+
+    auto *input_args = new (std::nothrow) fuse_read_in();
+    auto *file_data = reinterpret_cast<virtiofs_file_data*>(fp->f_data);
+    input_args->fh = file_data->file_handle;
+    input_args->offset = uio->uio_offset;
+    input_args->size = read_amt;
+    input_args->flags = ioflag;
+    input_args->lock_owner = 0;
+
+    virtiofs_debug("inode %d, reading %d bytes at offset %d\n", inode->nodeid, 
read_amt, uio->uio_offset);
+
+    auto *strategy = reinterpret_cast<fuse_strategy*>(vnode->v_mount->m_data);
+    auto error = fuse_req_send_and_receive_reply(strategy, FUSE_READ, 
inode->nodeid,
+            input_args, sizeof(*input_args), buf, read_amt);
+
+    int ret = 0;
+    if (!error) {
+        ret = uiomove(buf, read_amt, uio);
+    } else {
+        kprintf("[virtiofs] Error reading data\n");
+        ret = error;
+    }
+
+    free(buf);
+    free(input_args);
+
+    return ret;
+}
+//
+static int virtiofs_readdir(struct vnode *vnode, struct file *fp, struct 
dirent *dir)
+{
+    //TODO Implement
+    return EPERM;
+}
+
+static int virtiofs_getattr(struct vnode *vnode, struct vattr *attr)
+{
+    struct virtiofs_inode *inode = (struct virtiofs_inode *) vnode->v_data;
+
+    attr->va_mode = 0555; //Is it really correct?
+
+    if (S_ISDIR(inode->attr.mode)) {
+        attr->va_type = VDIR;
+    } else if (S_ISREG(inode->attr.mode)) {
+        attr->va_type = VREG;
+    } else if (S_ISLNK(inode->attr.mode)) {
+        attr->va_type = VLNK;
+    }
+
+    attr->va_nodeid = vnode->v_ino;
+    attr->va_size = inode->attr.size;
+
+    return 0;
+}
+
+#define virtiofs_write       ((vnop_write_t)vop_erofs)
+#define virtiofs_seek        ((vnop_seek_t)vop_nullop)
+#define virtiofs_ioctl       ((vnop_ioctl_t)vop_nullop)
+#define virtiofs_create      ((vnop_create_t)vop_erofs)
+#define virtiofs_remove      ((vnop_remove_t)vop_erofs)
+#define virtiofs_rename      ((vnop_rename_t)vop_erofs)
+#define virtiofs_mkdir       ((vnop_mkdir_t)vop_erofs)
+#define virtiofs_rmdir       ((vnop_rmdir_t)vop_erofs)
+#define virtiofs_setattr     ((vnop_setattr_t)vop_erofs)
+#define virtiofs_inactive    ((vnop_inactive_t)vop_nullop)
+#define virtiofs_truncate    ((vnop_truncate_t)vop_erofs)
+#define virtiofs_link        ((vnop_link_t)vop_erofs)
+#define virtiofs_arc         ((vnop_cache_t) nullptr)
+#define virtiofs_fallocate   ((vnop_fallocate_t)vop_erofs)
+#define virtiofs_fsync       ((vnop_fsync_t)vop_nullop)
+#define virtiofs_symlink     ((vnop_symlink_t)vop_erofs)
+
+struct vnops virtiofs_vnops = {
+    virtiofs_open,      /* open */
+    virtiofs_close,     /* close */
+    virtiofs_read,      /* read */
+    virtiofs_write,     /* write - returns error when called */
+    virtiofs_seek,      /* seek */
+    virtiofs_ioctl,     /* ioctl */
+    virtiofs_fsync,     /* fsync */
+    virtiofs_readdir,   /* readdir */
+    virtiofs_lookup,    /* lookup */
+    virtiofs_create,    /* create - returns error when called */
+    virtiofs_remove,    /* remove - returns error when called */
+    virtiofs_rename,    /* rename - returns error when called */
+    virtiofs_mkdir,     /* mkdir - returns error when called */
+    virtiofs_rmdir,     /* rmdir - returns error when called */
+    virtiofs_getattr,   /* getattr */
+    virtiofs_setattr,   /* setattr - returns error when called */
+    virtiofs_inactive,  /* inactive */
+    virtiofs_truncate,  /* truncate - returns error when called*/
+    virtiofs_link,      /* link - returns error when called*/
+    virtiofs_arc,       /* arc */ //TODO: Implement to allow memory re-use 
when mapping files, investigate using virtio-fs DAX
+    virtiofs_fallocate, /* fallocate - returns error when called*/
+    virtiofs_readlink,  /* read link */
+    virtiofs_symlink    /* symbolic link - returns error when called*/
+};
diff --git a/include/osv/device.h b/include/osv/device.h
index 16d2e470..17490bde 100755
--- a/include/osv/device.h
+++ b/include/osv/device.h
@@ -89,6 +89,7 @@ struct devops {
 #define        no_write        ((devop_write_t)enodev)
 #define        no_ioctl        ((devop_ioctl_t)enodev)
 #define        no_devctl       ((devop_devctl_t)nullop)
+#define        no_strategy     ((devop_strategy_t)nullop)
 
 /*
  * Driver object
-- 
2.20.1

-- 
You received this message because you are subscribed to the Google Groups "OSv 
Development" group.
To unsubscribe from this group and stop receiving emails from it, send an email 
to osv-dev+unsubscr...@googlegroups.com.
To view this discussion on the web visit 
https://groups.google.com/d/msgid/osv-dev/20200208232200.15009-2-jwkozaczuk%40gmail.com.

Reply via email to