gramineproject / gramine

A library OS for Linux multi-process applications, with Intel SGX support
GNU Lesser General Public License v3.0
601 stars 200 forks source link

[LibOS] Missing support for flock syscall #58

Closed trusty-fox closed 1 year ago

trusty-fox commented 4 years ago

Hello, I don't know if there is a better place to track this, but would it be possible to add support for the flock() system call in Graphene LibOS?

At the moment, in shim_syscalls.c this is declared as unsupported.

mkow commented 4 years ago

I think this is blocked on filesystem rewrite (the current filesystem code is really terrible :) ). Is there any particular application which doesn't work without flock()?

trusty-fox commented 4 years ago

I'm currently running PHP-CGI in Graphene (with and without SGX) and the PHP web applications I want to run make use of different ways of file locking. I can make it work by disabling all locking, but I don't think this is a useful workaround in practise.

Do you know any plans concerning rewriting the filesystem layer?

mkow commented 4 years ago

We have plans to do it, but no one has started this work yet. I hope to do it sooner than later as this is stopping us from progressing in a few areas and it's getting more and more annoying ;)

dimakuv commented 4 years ago

+1 to Michal's remark about the rewrite of the FS subsystem before implementing flock().

Btw, isn't flock() (lock the file) is kinda useless in Graphene-SGX? We don't trust the host OS to actually honor file locks. The only thing that can be done is to restrict simultaneous file accesses when both originate from Graphene-SGX (i.e., in two threads). Though I think that this scenario is very improbable. In the end, maybe it's just a matter of always returning success and having a warning in Graphene...

mkow commented 4 years ago

I think we could have correct locking in protected fs and tmpfs? Only for chrootfs it's not possible.

dimakuv commented 4 years ago

At the LibOS level, there is no "protected fs" (it is implemented purely at the Linux-SGX PAL layer).

For tmpfs sure. We need tmpfs first though :)

mkow commented 4 years ago

btw., related issue: gramineproject/graphene#437

mkow commented 3 years ago

Dropping this to P2, as I think we don't know any workload which doesn't have a fallback to POSIX locks (which were added recently).

dimakuv commented 2 years ago

Looking at this from August 2022, some things are irrelevant/were done already:

I think this is blocked on filesystem rewrite (the current filesystem code is really terrible :) ).

The FS rewrite was done.

At the LibOS level, there is no "protected fs" (it is implemented purely at the Linux-SGX PAL layer).

Protected FS (renamed to a more logical Encrypted FS) was lifted from the PAL layer to the LibOS level.

For tmpfs sure. We need tmpfs first though :)

Tmpfs was implemented.

aep commented 2 years ago

would everyone be ok with just ignoring the syscall and returning 0 ?

i don't think there's value in actually implementing it inside tmpfs for example because there's no second process that contests it anyway.

could be made a config option, i guess.

boryspoplawski commented 2 years ago

would everyone be ok with just ignoring the syscall and returning 0 ?

Definitely not, if something depends on it, it would break horribly.

there's no second process that contests it anyway.

There could be multiple threads (which share virtual memory and hence tmpfs)

dimakuv commented 2 years ago

+1 to @boryspoplawski, we should implement flock() properly.

fnerdman commented 2 years ago

We've run into this problem graminizing geth. As this issue has seen some activity recently, I am wondering if a patch is already being developed now or in the near future?

dimakuv commented 2 years ago

@lead4good We'll discuss whether we have the manpower to work on flock(). We just finished v1.3, so maybe we can include it in v1.4... No promises though.

fnerdman commented 2 years ago

@dimakuv Great news! Please let us know once there is a final decision on whether a flock implementation will be included in v1.4 or not.

fnerdman commented 2 years ago

For our use case, implementing flock for tmpfs would be sufficient for now. As tmpfs is per process only my understanding would be we could always return success?

fnerdman commented 2 years ago

@dimakuv @boryspoplawski Would you accept a PR that implements flock for tmpfs only by always returning success?

boryspoplawski commented 2 years ago

@lead4good No, as it was already discussed above.

fnerdman commented 2 years ago

@boryspoplawski

There could be multiple threads (which share virtual memory and hence tmpfs)

From my understanding reading the flock manpage, file locking happens at the process level. So multiple threads of the same process should be able to successfully flock at the same time. Returning success for tmpfs files which exist per process should be the same behaviour then, right?

boryspoplawski commented 2 years ago

How about instead of guessing you check the actual behavior?

qijiax commented 1 year ago

Same issue in running TDengine an open-source time series database.

dimakuv commented 1 year ago

I started reading about flock(). Here are the materials I found particularly interesting and enlightening:

dimakuv commented 1 year ago

I feel like this important difference between fcntl() and flock() must be highlighted (in Gramine terms):

dimakuv commented 1 year ago

Just FYI, since the commit that introduced POSIX locks (fcntl(F_SETLK)) is hard to find. Here is the commit: https://github.com/gramineproject/gramine/commit/abf44e9febdbe3da19a0147f61685baf2edfd264

I'm going through our current implementation of POSIX locks, to understand what could be the possible design on flock() in Gramine.

dimakuv commented 1 year ago

Ok, so here is my proposal for implementing flock():

There must be extensive testing for this:

Also, if we do not support fork for flock locks, we should mention it very explicitly somewhere.


UPDATE: I don't like my initial proposal of posix_lock::handle = <pointer-in-gramine-process>. This is an ugly hack and it will not be extensible to forking. So instead, to support fork, I propose the following:

dimakuv commented 1 year ago

Another quick note: in Gramine, we don't have the mapping (process, inode) -> lock state, instead we have the mapping (process, filename) -> lock state (filename is actually a dentry, but the dentry is found through filename lookup, so in the general case it's just the filename).

This is why e.g. Gramine cannot support file locking on tmpfs at all (except for in master process itself): the master will never lookup a filename of such a tmpfs file.

dimakuv commented 1 year ago

@pwmarcz If you still remember this work and have some time, could you give any feedback on my above proposal for implementing flock()?

@kailun-qin You may be also interested in my findings.

pwmarcz commented 1 year ago

@dimakuv

I agree, having an explicit handle ID is much better than using a pointer value. Maybe you could use PID of the process that created the handle as part of handle ID? This would allow you to assigns handle IDs without using any IPC to ensure uniqueness. E.g. handle_id = (pid << 32) | counter. I assume you want to store the handle ID inside handle, and the child process would keep the handle ID created by the parent.

I agree with the part about reusing infrastructure for POSIX locks, it does make things simpler - there is a lot of boilerplate.

The rest of the proposal sounds fine, according to what I remember about POSIX locks.

billionairiam commented 1 year ago

Below is my implementation of flock:

  1. Add posix_lock::handle_id into struct posix_lock, which is transferred by flock's argument fd, but in gramine represents handle id.
  2. Update posix_lock_find_conflict function. when called flock, we assign fd to posix_lock::handle_id. posix_lock_find_conflict use posix_lock::handle_id to filter conflicts.

    static struct posix_lock* posix_lock_find_conflict(struct fs_lock* fs_lock, struct posix_lock* pl) {
    assert(locked(&g_fs_lock_lock));
    assert(pl->type != F_UNLCK);
    
    struct posix_lock* cur;
    /*Called by fcntl*/
    if (pl->fd == -1) {
        LISTP_FOR_EACH_ENTRY(cur, &fs_lock->posix_locks, list) {
        if (cur->pid != pl->pid && pl->start <= cur->end && cur->start <= pl->end
               && (cur->type == F_WRLCK || pl->type == F_WRLCK))
            return cur;
        }
    /*Called by flock*/
    } else {
        LISTP_FOR_EACH_ENTRY(cur, &fs_lock->posix_locks, list) {
        if (cur->fd != pl->fd && (cur->type == F_WRLCK || pl->type == F_WRLCK))
            return cur;
        }
    }
    
    return NULL;
    }
  3. As flock only lock the whole file and a shared or exclusive lock can be placed on a file regardless of the mode in which the file was opened. so I define flock's entry function like this:

    
    long libos_syscall_flock(int fd, int operation) {
    int ret;
    
    struct libos_handle_map* handle_map = get_thread_handle_map(NULL);
    assert(handle_map);
    
    struct libos_handle* hdl = get_fd_handle(fd, NULL, handle_map);
    if (!hdl)
        return -EBADF;
    
    struct flock fl;
    
    switch (operation & ~LOCK_NB) {
        case LOCK_EX:
            fl.l_type = F_WRLCK;
            break;
        case LOCK_SH:
            fl.l_type = F_RDLCK;
            break;
        case LOCK_UN:
            fl.l_type = F_UNLCK;
            break;
        default:
            ret = -EINVAL;
            goto out;
    }
    
    fl.l_whence = SEEK_SET;
    fl.l_start = fl.l_len = 0L;/* Lock the whold file. */
    
    struct posix_lock pl;
    ret = flock_to_posix_lock(&fl, hdl, &pl);
    if (ret < 0)
        goto out;
    
    pl.handle_id = fd;
    if (fl.l_type == F_UNLCK) {
        ret = posix_lock_clear_hid(fd);
        if (ret)
            goto out;
    }
    
    ret = posix_lock_set(hdl->dentry, &pl, !(operation&LOCK_NB));

out: put_handle(hdl); return ret; }


[`posix_lock_set`](https://github.com/gramineproject/gramine/blob/da7ea3f0f4e72033ccbea877e5599ee65564a516/libos/src/fs/libos_fs_lock.c#L441) is shared between `fcntl` and `flock`. but add `handle_id` to `posix_lock` when doing[ `_posix_lock_set`](https://github.com/gramineproject/gramine/blob/da7ea3f0f4e72033ccbea877e5599ee65564a516/libos/src/fs/libos_fs_lock.c#L190)

4. when unlocking a file of a target `handle_id`, if it is the leader process, we should remove `posix_lock` and `posix_lock_request` from `fs_lock`. but if it is the child process, we should send a message to IPC, and let the leader process do this operation.
5. When `LOCK_NB` is not set and conflicts are founded, a `posix_lock_request` should add to `fs_lock`. But in the child process, an IPC message should be sent to let the leader do this operation.
kailun-qin commented 1 year ago

we assign fd to posix_lock::handle_id

@billionairiam Hi Liang, as I commented in the team's review earlier yesterday, using fd directly as posix_lock::handle_id is not right. Pls kindly take a look at the proposal here: https://github.com/gramineproject/gramine/issues/58#issuecomment-1437453697.

dimakuv commented 1 year ago

@liangintel At a quick glance, the implementation looks correct. Except the usage of fd as the "flock ID". Consider the following snippet:

    int fd1 = open("myfile");
    flock(fd1, LOCK_EX);
    ... read(fd1) ...
    int fd2 = dup(fd1);
    flock(fd2, LOCK_UN);

In your current implementation, fd1 and fd2 will be considered by Gramine as two different locks. But in reality they are the same lock, so this code snippet should result in first locking the file, and then unlocking this file.

Or even more dramatic example:

    int fd1 = open("myfile");
    flock(fd1, LOCK_EX);
    ... read(fd1) ...
    int fd2 = dup(fd1);
    flock(fd2, LOCK_EX);
    flock(fd1, LOCK_UN);

What's gonna happen here? In your implementation, flock(fd2, LOCK_EX) will hang the application forever. In normal Linux, flock(fd2, LOCK_EX) will be treated as a no-op (because the same lock is acquired again, which does nothing).

Please also write several tests for flock() under libos/test/regression. And enable as many LTP flock tests as possible.

liangintel commented 1 year ago

seems the expected person to at is @billionairiam Liang Ma, not @liangintel Liang Fang :)

dimakuv commented 1 year ago

Oops, sorry. Yes, the expected person is @billionairiam :)

billionairiam commented 1 year ago

@liangintel At a quick glance, the implementation looks correct. Except the usage of fd as the "flock ID". Consider the following snippet:

    int fd1 = open("myfile");
    flock(fd1, LOCK_EX);
    ... read(fd1) ...
    int fd2 = dup(fd1);
    flock(fd2, LOCK_UN);

In your current implementation, fd1 and fd2 will be considered by Gramine as two different locks. But in reality they are the same lock, so this code snippet should result in first locking the file, and then unlocking this file.

Or even more dramatic example:

    int fd1 = open("myfile");
    flock(fd1, LOCK_EX);
    ... read(fd1) ...
    int fd2 = dup(fd1);
    flock(fd2, LOCK_EX);
    flock(fd1, LOCK_UN);

What's gonna happen here? In your implementation, flock(fd2, LOCK_EX) will hang the application forever. In normal Linux, flock(fd2, LOCK_EX) will be treated as a no-op (because the same lock is acquired again, which does nothing).

Please also write several tests for flock() under libos/test/regression. And enable as many LTP flock tests as possible.

Yes, Working on solving the dup situation. Going to use the pointer hanlde and handle_id to solve this situation. Why need pointer hanlde because we need to distinguish between dup and open. Only using handle_id seems not enough.

billionairiam commented 1 year ago

After discussing with @kailun-qin, Maybe no need to add the field handle into posix_lock to keep the neatness.

dimakuv commented 1 year ago

@billionairiam I would recommend to add the field uint32_t id into libos_handle struct, somewhere here: https://github.com/gramineproject/gramine/blob/38aa937772c8f6d74d94f7d84dddf2cd3e2c2c54/libos/include/libos_handle.h#L152

And you'll set this field to an ever-increasing number somewhere here: https://github.com/gramineproject/gramine/blob/38aa937772c8f6d74d94f7d84dddf2cd3e2c2c54/libos/src/bookkeep/libos_handle.c#L365

Also note that you don't need to do anything special for the fork (checkpoint-and-restore) logic, because this new field id will automatically be checkpointed and restored in the child process after fork: https://github.com/gramineproject/gramine/blob/38aa937772c8f6d74d94f7d84dddf2cd3e2c2c54/libos/src/bookkeep/libos_handle.c#L821

Also, you already introduced the field uint64_t posix_lock::handle_id. So whenever you assign to this field, you can do like this:

    posix_lock->handle_id = ((uint64_t)g_process.pid << 32) | libos_handle->id;

As @pwmarcz explained, this guarantees that posix_lock::handle_id is globally unique across all Gramine processes.

After discussing with @kailun-qin, Maybe no need to add the field handle into posix_lock to keep the neatness.

I'm not sure what you mean here, I was actually expecting to have this... But maybe you're right. Please give the details of what you mean by this.

kailun-qin commented 1 year ago

I'm not sure what you mean here, I was actually expecting to have this... But maybe you're right. Please give the details of what you mean by this.

I guess @billionairiam was commenting on his previous comment (https://github.com/gramineproject/gramine/issues/58#issuecomment-1447932849, also copied below), where he was trying to solve the dup() case by introducing both a pointer value to hanlde and handle_id into posix_lock (i.e., two fields). But then after some discussions he realized that handle_id would be enough to act as the handle identifier (i.e., the pointer to handle is hence not needed). Anyway, I'll let him to confirm.

Yes, Working on solving the dup situation. Going to use the pointer hanlde and handle_id to solve this situation. Why need pointer hanlde because we need to distinguish between dup and open. Only using handle_id seems not enough.

pwmarcz commented 1 year ago

@dimakuv

posix_lock->handle_id = ((uint64_t)g_process.pid << 32) | libos_handle->id; As @pwmarcz explained, this guarantees that posix_lock::handle_id is globally unique across all Gramine processes.

I don't think that's correct. If a handle is inherited from the parent process via fork, and both parent and child take a lock, we want them to end up with the same posix lock ID, right? Which would mean that the handle id should include the PID of process that originally created the handle.

dimakuv commented 1 year ago

If a handle is inherited from the parent process via fork, and both parent and child take a lock, we want them to end up with the same posix lock ID, right? Which would mean that the handle id should include the PID of process that originally created the handle.

Yes. But I wanted for the posix_lock object to also be checkpointed-and-restored. Yeah, I wasn't clear about this part in my design description, I see now (I didn't mention that part).

So the posix_lock object will be restored in the child, and its posix_lock::handle_id will be kept the same as in the parent. @pwmarcz Does that make sense, or did I miss something again?

pwmarcz commented 1 year ago

@dimakuv I don't understand how that would work in the scenario I described: first we fork, and then two processes take a lock using "the same" handle.

My point is: this unique identifier has to be created on handle creation, even before any kind of lock is taken. I think the easiest way to achieve that is making libos_handle->id unique between processes.

(And I don't think we need to store any other information (such as struct posix_lock) in processes other than main. These processes do not actually know if there's a lock associated with a given handle, precisely because a different process with a copy of the same original handle could have taken it.)

dimakuv commented 1 year ago

My point is: this unique identifier has to be created on handle creation, even before any kind of lock is taken. I think the easiest way to achieve that is making libos_handle->id unique between processes.

I get it now. You are right.

So actually it should be:

libos_handle->id = ((uint64_t)g_process.pid << 32) | local_counter;

And posix_lock object is created with this libos_handle->id as an identifier (for flock case).

billionairiam commented 1 year ago

The key point is to give a fd's libos_handle a unique ID that is unvary cross fork.

kailun-qin commented 1 year ago

The key point is to give a fd's libos_handle a unique ID that is unvary cross fork.

The ID will be kept the same using this proposal of creating libos_handle->id = ((uint64_t)g_process.pid << 32) | local_counter; on handle creation (per our discussions offline).

billionairiam commented 1 year ago
  1. To solve dup case, in libos_open.c add static uint64_t local_counter = 0; after get_new_handle we assign hdl->ID = ((uint64_t)g_process.pid << 32) | local_counter++; should notice overflow here.
  2. In libos_syscall_close the (void)clear_posix_locks(handle) was called in detach_fd_handle which didn't check the handle's ref_count. so I add a field in posix_lock:

    DEFINE_LISTP(posix_lock);
    DEFINE_LIST(posix_lock);
    struct posix_lock {
    /* Lock type: F_RDLCK, F_WRLCK, F_UNLCK */
    int type;
    
    /* First byte of range */
    uint64_t start;
    
    /* Last byte of range (use FS_LOCK_EOF for a range until end of file) */
    uint64_t end;
    
    /* PID of process taking the lock */
    IDTYPE pid;
    
    /* List node, used internally */
    LIST_TYPE(posix_lock) list;
    
    /* Related handle id */
    uint64_t handle_id;
    
    /* Related `libos_handle` ref_count */
    int64_t ref_count;
    };
  3. The posix_lock_find_conflict now uses handle_id to distinguish fcntl from flock.

    static struct posix_lock* posix_lock_find_conflict(struct fs_lock* fs_lock, struct posix_lock* pl) {
    assert(locked(&g_fs_lock_lock));
    assert(pl->type != F_UNLCK);
    
    struct posix_lock* cur;
    if (pl->handle_id == 0) {
        LISTP_FOR_EACH_ENTRY(cur, &fs_lock->posix_locks, list) {
            if (cur->pid != pl->pid && pl->start <= cur->end && cur->start <= pl->end
                    && (cur->type == F_WRLCK || pl->type == F_WRLCK))
                return cur;
        }
    } else {
        LISTP_FOR_EACH_ENTRY(cur, &fs_lock->posix_locks, list) {
            if (cur->handle_id != pl->handle_id && (cur->type == F_WRLCK || pl->type == F_WRLCK))
                return cur;
        }
    }
    return NULL;
    }
  4. the _posix_lock_set also use handle_id to distinguish fcntl from flock.

    static int _posix_lock_set(struct fs_lock* fs_lock, struct posix_lock* pl) {
    assert(locked(&g_fs_lock_lock));
    
    if ((&fs_lock->posix_locks) && ((&fs_lock->posix_locks)->first) && (&fs_lock->posix_locks)->first->handle_id == 0) {
        pl->handle_id = 0;
    }
    /* Preallocate new locks first, so that we don't fail after modifying something. */
    
    /* Lock to be added. Not necessary for F_UNLCK, because we're only removing existing locks. */
    struct posix_lock* new = NULL;
    if (pl->type != F_UNLCK) {
        new = malloc(sizeof(*new));
        if (!new)
            return -ENOMEM;
    }
    
    /* Extra lock that we might need when splitting existing one. */
    struct posix_lock* extra = malloc(sizeof(*extra));
    if (!extra) {
        free(new);
        return -ENOMEM;
    }
    
    /* Target range: we will be changing it when merging existing locks. */
    uint64_t start = pl->start;
    uint64_t end   = pl->end;
    
    /* `prev` will be set to the last lock before target range, so that we add the new lock just
     * after `prev`. */
    struct posix_lock* prev = NULL;
    
    struct posix_lock* cur;
    struct posix_lock* tmp;
    if (pl->handle_id == 0) {
        LISTP_FOR_EACH_ENTRY_SAFE(cur, tmp, &fs_lock->posix_locks, list) {
            if (cur->pid < pl->pid) {
                prev = cur;
                continue;
            }
            if (pl->pid < cur->pid) {
                break;
            }
    
            if (cur->type == pl->type) {
                /* Same lock type: we can possibly merge the locks. */
    
                if (start > 0 && cur->end < start - 1) {
                    /* `cur` ends before target range begins, and is not even adjacent */
                    prev = cur;
                } else if (end < FS_LOCK_EOF && end + 1 < cur->start) {
                    /* `cur` begins after target range ends, and is not even adjacent - we're
                    * done */
                    break;
                } else {
                    /* `cur` is either adjacent to target range, or overlaps with it. Delete it, and
                    * expand the target range. */
                    start = MIN(start, cur->start);
                    end = MAX(end, cur->end);
                    LISTP_DEL(cur, &fs_lock->posix_locks, list);
                    free(cur);
                }
            } else {
                /* Different lock types: if they overlap, we delete the target range. */
    
                if (cur->end < start) {
                    /* `cur` ends before target range begins */
                    prev = cur;
                } else if (end < cur->start) {
                    /* `cur` begins after target range ends - we're done */
                    break;
                } else if (cur->start < start && cur->end <= end) {
                    /*
                    * `cur` overlaps with beginning of target range. Shorten `cur`.
                    *
                    * cur:  =======
                    * tgt:    -------
                    *
                    * cur:  ==
                    */
                    assert(start > 0);
                    cur->end = start - 1;
                    prev = cur;
                } else if (cur->start < start && cur->end > end) {
                    /*
                    * The target range is inside `cur`. Split `cur` and finish.
                    *
                    * cur:    ========
                    * tgt:      ----
                    *
                    * cur:    ==
                    * extra:        ==
                    */
    
                    /* We'll need `extra` only once, because we exit the loop afterwards. */
                    assert(extra);
    
                    assert(start > 0);
                    extra->type = cur->type;
                    extra->start = end + 1;
                    extra->end = cur->end;
                    extra->pid = cur->pid;
                    cur->end = start - 1;
                    LISTP_ADD_AFTER(extra, cur, &fs_lock->posix_locks, list);
                    extra = NULL;
                    /* We're done: the new lock, if any, will be added after `cur`. */
                    prev = cur;
                    break;
                } else if (start <= cur->start && cur->end <= end) {
                    /*
                    * `cur` is completely covered by target range. Delete `cur`.
                    *
                    * cur:    ====
                    * tgt:  --------
                    */
                    LISTP_DEL(cur, &fs_lock->posix_locks, list);
                    free(cur);
                } else {
                    /*
                    * `cur` overlaps with end of target range. Shorten `cur` and finish.
                    *
                    * cur:    ====
                    * tgt: -----
                    *
                    * cur:      ==
                    */
                    assert(start <= cur->start && end < cur->end);
                    assert(end < FS_LOCK_EOF);
                    cur->start = end + 1;
                    break;
                }
            }
        }
    } else {
        LISTP_FOR_EACH_ENTRY_SAFE(cur, tmp, &fs_lock->posix_locks, list) {
            if (cur->handle_id == pl->handle_id) {
                if (pl->type != F_UNLCK) {
                    LISTP_DEL(cur, &fs_lock->posix_locks, list);
                    free(cur);
                    break;
                } else {
                    cur->ref_count--;
                    if (cur->ref_count == 0) {
                        LISTP_DEL(cur, &fs_lock->posix_locks, list);
                        free(cur);
                        break;
                    }
                }
            }
        }
    }
    
    if (new) {
        assert(pl->type != F_UNLCK);
    
        new->type = pl->type;
        new->start = start;
        new->end = end;
        new->pid = pl->pid;
        new->handle_id = pl->handle_id;
        new->ref_count = pl->ref_count;
    #ifdef DEBUG
        /* Assert that list order is preserved */
        struct posix_lock* next = prev ? LISTP_NEXT_ENTRY(prev, &fs_lock->posix_locks, list)
            : LISTP_FIRST_ENTRY(&fs_lock->posix_locks, struct posix_lock, list);
        if (prev)
            assert(prev->pid < pl->pid || (prev->pid == pl->pid && prev->end < start));
        if (next)
            assert(pl->pid < next->pid || (pl->pid == next->pid && end < next->start));
    #endif
    
        if (prev) {
            LISTP_ADD_AFTER(new, prev, &fs_lock->posix_locks, list);
        } else {
            LISTP_ADD(new, &fs_lock->posix_locks, list);
        }
    }
    
    if (extra)
        free(extra);
    return 0;
    }

    when we close a handle, the clear_posix_locks use _posix_lock_set to clear posix_lock or just ref_count--.

    static int clear_posix_locks(struct libos_handle* handle) {
    if (handle && handle->dentry) {
        /* Clear POSIX locks for a file. We are required to do that every time a FD is closed, even
         * if the process holds other handles for that file, or duplicated FDs for the same
         * handle. */
        struct posix_lock pl = {
            .type = F_UNLCK,
            .start = 0,
            .end = FS_LOCK_EOF,
            .pid = g_process.pid,
            .handle_id = handle->ID,
        };
        int ret = posix_lock_set(handle->dentry, &pl, /*block=*/false);
        if (ret < 0) {
            log_warning("error releasing locks: %s", unix_strerror(ret));
            return ret;
        }
    }
    
    return 0;
    }
  5. The libos_syscall_flock now like this:

    
    long libos_syscall_flock(int fd, int operation) {
    int ret;
    
    struct libos_handle_map* handle_map = get_thread_handle_map(NULL);
    assert(handle_map);
    
    struct libos_handle* hdl = get_fd_handle(fd, NULL, handle_map);
    if (!hdl)
        return -EBADF;
    
    struct flock fl;
    
    switch (operation & ~LOCK_NB) {
        case LOCK_EX:
            fl.l_type = F_WRLCK;
            break;
        case LOCK_SH:
            fl.l_type = F_RDLCK;
            break;
        case LOCK_UN:
            fl.l_type = F_UNLCK;
            break;
        default:
            ret = -EINVAL;
            goto out;
    }
    
    fl.l_whence = SEEK_SET;
    fl.l_start = fl.l_len = 0L;/* Lock the whold file. */
    
    struct posix_lock pl;
    ret = flock_to_posix_lock(&fl, hdl, &pl);
    if (ret < 0)
        goto out;
    
    pl.handle_id = (uint64_t)hdl->ID;
    pl.ref_count = hdl->ref_count;
    
    if (fl.l_type == F_UNLCK) {
        ret = posix_lock_clear_hid(pl.handle_id);
        goto out;
    }
    
    ret = posix_lock_set(hdl->dentry, &pl, !(operation&LOCK_NB));

out: put_handle(hdl); return ret;

}


6. Now the `posix_locks` inside `fs_lock` doesn't sort by `pid` for `flock`.
dimakuv commented 1 year ago

In libos_syscall_close the (void)clear_posix_locks(handle) was called in detach_fd_handle which didn't check the handle's ref_count. so I add a field in posix_lock

This sounds wrong. If I understand you correctly, you now broke the case of fcntl locks. But we must support both fcntl locks' logic (which unlocks the fcntl lock on each close(fd)) and flock locks' logic (which unlocks the flock lock only on the very last close(fd)).

So you'll actually need to distinguish the two cases during libos_syscall_close() somehow.

dimakuv commented 1 year ago

The rest of @billionairiam's points (1, 3, 4, 5, 6) seem Ok to me, but I haven't read in detail.

The only small comment: please don't use capital letters for variable names. So please change libos_handle::ID to libos_handle::id.

billionairiam commented 1 year ago

In libos_syscall_close the (void)clear_posix_locks(handle) was called in detach_fd_handle which didn't check the handle's ref_count. so I add a field in posix_lock

This sounds wrong. If I understand you correctly, you now broke the case of fcntl locks. But we must support both fcntl locks' logic (which unlocks the fcntl lock on each close(fd)) and flock locks' logic (which unlocks the flock lock only on the very last close(fd)).

So you'll actually need to distinguish the two cases during libos_syscall_close() somehow.

Yes I didn't break fcntl's close logic, just added the field handle_id into the posix_lock in function (void)clear_posix_locks(handle):

static int clear_posix_locks(struct libos_handle* handle) {
    if (handle && handle->dentry) {
        /* Clear POSIX locks for a file. We are required to do that every time a FD is closed, even
         * if the process holds other handles for that file, or duplicated FDs for the same
         * handle. */
        struct posix_lock pl = {
            .type = F_UNLCK,
            .start = 0,
            .end = FS_LOCK_EOF,
            .pid = g_process.pid,
            //Added here also changed ID->id
            .handle_id = handle->id,
        };
        int ret = posix_lock_set(handle->dentry, &pl, /*block=*/false);
        if (ret < 0) {
            log_warning("error releasing locks: %s", unix_strerror(ret));
            return ret;
        }
    }

    return 0;
}
dimakuv commented 1 year ago

@billionairiam I still don't understand. So you call this clear_posix_locks() on every detach_fd_handle(), which means on every close of the file descriptor. And inside this function, you call posix_lock_set(... F_UNLCK ...). So how is ref_count relevant here? Looks like you'll send the unlock request every time an FD is closed.

billionairiam commented 1 year ago

@billionairiam I still don't understand. So you call this clear_posix_locks() on every detach_fd_handle(), which means on every close of the file descriptor. And inside this function, you call posix_lock_set(... F_UNLCK ...). So how is ref_count relevant here? Looks like you'll send the unlock request every time an FD is closed.

The gramine's original code is here. If I'am not misunderstand that the clear_posix_locks is called in every close.

billionairiam commented 1 year ago

@billionairiam I still don't understand. So you call this clear_posix_locks() on every detach_fd_handle(), which means on every close of the file descriptor. And inside this function, you call posix_lock_set(... F_UNLCK ...). So how is ref_count relevant here? Looks like you'll send the unlock request every time an FD is closed.

I understand it now, remove ref_count from posix_lock and check whether we need to call clear_posix_locks inside detach_fd_handle.

struct libos_handle* detach_fd_handle(uint32_t fd, int* flags,
                                      struct libos_handle_map* handle_map) {
    struct libos_handle* handle = NULL;

    if (!handle_map && !(handle_map = get_thread_handle_map(NULL)))
        return NULL;

    lock(&handle_map->lock);

    if (fd < handle_map->fd_size)
        handle = __detach_fd_handle(handle_map->map[fd], flags, handle_map);

    unlock(&handle_map->lock);
    if (handle->id == 0 || (handle->ref_count ==1 && handle->id != 0)) {
        (void)clear_posix_locks(handle);
    }

    return handle;
}

if handle->id == 0 now it's fcntl case. if handle->ref_count ==1 && handle->id != 0 it's flock case also with all such descriptors have been closed.