/*
* CDDL HEADER START
*
* The contents of this file are subject to the terms of the
* Common Development and Distribution License (the "License").
* You may not use this file except in compliance with the License.
*
* You can obtain a copy of the license at usr/src/OPENSOLARIS.LICENSE
* See the License for the specific language governing permissions
* and limitations under the License.
*
* When distributing Covered Code, include this CDDL HEADER in each
* file and include the License file at usr/src/OPENSOLARIS.LICENSE.
* If applicable, add the following below this CDDL HEADER, with the
* fields enclosed by brackets "[]" replaced with your own identifying
* information: Portions Copyright [yyyy] [name of copyright owner]
*
* CDDL HEADER END
*/
/*
* Copyright (c) 2011, 2016 by Delphix. All rights reserved.
* Copyright (c) 2014 Integros [integros.com]
*/
/* Portions Copyright 2010 Robert Milkowski */
#include <sys/zfs_context.h>
#include <sys/resource.h>
#include <sys/zil_impl.h>
#include <sys/dsl_dataset.h>
#include <sys/vdev_impl.h>
#include <sys/dsl_pool.h>
/*
* The zfs intent log (ZIL) saves transaction records of system calls
* that change the file system in memory with enough information
* to be able to replay them. These are stored in memory until
* either the DMU transaction group (txg) commits them to the stable pool
* and they can be discarded, or they are flushed to the stable log
* (also in the pool) due to a fsync, O_DSYNC or other synchronous
* requirement. In the event of a panic or power fail then those log
* records (transactions) are replayed.
*
* There is one ZIL per file system. Its on-disk (pool) format consists
* of 3 parts:
*
* - ZIL header
* - ZIL blocks
* - ZIL records
*
* A log record holds a system call transaction. Log blocks can
* hold many log records and the blocks are chained together.
* Each ZIL block contains a block pointer (blkptr_t) to the next
* ZIL block in the chain. The ZIL header points to the first
* block in the chain. Note there is not a fixed place in the pool
* to hold blocks. They are dynamically allocated and freed as
* needed from the blocks available. Figure X shows the ZIL structure:
*/
/*
* Disable intent logging replay. This global ZIL switch affects all pools.
*/
int zil_replay_disable = 0;
/*
* Tunable parameter for debugging or performance analysis. Setting
* zfs_nocacheflush will cause corruption on power loss if a volatile
* out-of-order write cache is enabled.
*/
/*
* ziltest is by and large an ugly hack, but very useful in
* checking replay without tedious work.
* When running ziltest we want to keep all itx's and so maintain
* a single list in the zl_itxg[] that uses a high txg: ZILTEST_TXG
* We subtract TXG_CONCURRENT_STATES to allow for common code.
*/
static int
{
return (-1);
return (1);
return (-1);
return (1);
return (0);
}
static void
{
}
static void
{
avl_destroy(t);
}
int
{
if (BP_IS_EMBEDDED(bp))
return (0);
return (0);
}
static zil_header_t *
{
}
static void
{
}
/*
* Read a log block and make sure it's valid.
*/
static int
char **end)
{
int error;
if (error == 0) {
/*
* Validate the checksummed log block.
*
* Sequence numbers should be... sequential. The checksum
* verifier for the next block should be bp's checksum plus 1.
*
* Also check the log chain linkage and size used.
*/
} else {
}
} else {
} else {
}
}
}
return (error);
}
/*
* Read a TX_WRITE log data block.
*/
static int
{
int error;
if (BP_IS_HOLE(bp)) {
return (0);
}
if (error == 0) {
}
return (error);
}
/*
* Parse the intent log, and call parse_func for each valid record within.
*/
int
{
int error = 0;
/*
* Old logs didn't record the maximum zh_claim_lr_seq.
*/
/*
* Starting at the block pointed to by zh_log we read the log chain.
* For each block in the chain we strongly check that block to
* ensure its validity. We stop when an invalid block is found.
* For each block pointer in the chain we call parse_blk_func().
* For each record in each valid block we call parse_lr_func().
* If the log has been claimed, stop if we encounter a sequence
* number greater than the highest claimed sequence number.
*/
int reclen;
char *end;
if (blk_seq > claim_blk_seq)
break;
break;
blk_count++;
break;
if (error != 0)
break;
goto done;
goto done;
lr_count++;
}
}
done:
return (error);
}
static int
{
/*
* Claim log block if not already committed and not already claimed.
* If tx == NULL, just verify that the block is claimable.
*/
return (0);
}
static int
{
int error;
return (0);
/*
* If the block is not readable, don't claim it. This can happen
* in normal operation when a log block is written to disk before
* some of the dmu_sync() blocks it points to. In this case, the
* transaction cannot have been committed to anyone (we would have
* waited for all writes to be stable first), so it is semantically
* correct to declare this the end of the log.
*/
return (error);
}
/* ARGSUSED */
static int
{
return (0);
}
static int
{
/*
* If we previously claimed it, we need to free it.
*/
!BP_IS_HOLE(bp))
return (0);
}
static lwb_t *
{
} else {
}
return (lwb);
}
/*
* Called when we create in-memory log transactions so that we know
* to cleanup the itxs at the end of spa_sync().
*/
void
{
if (ds->ds_is_snapshot)
panic("dirtying snapshot!");
/* up the hold count until we can be written out */
}
}
/*
* Determine if the zil is dirty in the specified txg. Callers wanting to
* ensure that the dirty state does not change must hold the itxg_lock for
* the specified txg. Holding the lock will ensure that the zil cannot be
* dirtied (zil_itx_assign) or cleaned (zil_clean) while we check its current
* state.
*/
{
return (B_TRUE);
return (B_FALSE);
}
/*
* Determine if the zil is dirty. The zil is considered dirty if it has
* any pending itx records that have not been cleaned by zil_clean().
*/
{
for (int t = 0; t < TXG_SIZE; t++) {
return (B_TRUE);
}
return (B_FALSE);
}
/*
* Create an on-disk intent log.
*/
static lwb_t *
{
int error = 0;
/*
* Wait for any previous destroy to complete.
*/
/*
* Allocate an initial log block if:
* - there isn't one already
* - the existing block is the wrong endianess
*/
if (!BP_IS_HOLE(&blk)) {
}
if (error == 0)
}
/*
* Allocate a log write buffer (lwb) for the first log block.
*/
if (error == 0)
/*
* If we just allocated the first log block, commit our transaction
* and wait for zil_sync() to stuff the block poiner into zh_log.
* (zh is part of the MOS, so we cannot modify it in open context.)
*/
}
return (lwb);
}
/*
* In one tx, free all log blocks and clear the log header.
* If keep_first is set, then we're replaying a log with no content.
* We want to keep the first block, however, so that the first
* synchronous transaction doesn't require a txg_wait_synced()
* in zil_create(). We don't need to txg_wait_synced() here either
* when keep_first is set, because both zil_create() and zil_destroy()
* will wait for any in-progress destroys to complete.
*/
void
{
/*
* Wait for any previous destroy to complete.
*/
return;
VERIFY(!keep_first);
}
} else if (!keep_first) {
}
}
void
{
}
int
{
int error;
if (error != 0) {
/*
* EBUSY indicates that the objset is inconsistent, in which
* case it can not have a ZIL.
*/
}
return (0);
}
return (0);
}
/*
* Claim all log blocks if we haven't already done so, and remember
* the highest claimed sequence number. This ensures that if we can
* read only part of the log now (e.g. due to a missing device),
* but we can read the entire log later, we will not try to replay
* or destroy beyond the last block we successfully claimed.
*/
}
return (0);
}
/*
* Check the log by walking the log chain.
* Checksum errors are ok as they indicate the end of the chain.
* Any other error (no device or read failure) returns an error.
*/
/* ARGSUSED */
int
{
int error;
if (error != 0) {
return (0);
}
/*
* Check the first block and determine if it's on a log device
* which may have been removed or faulted prior to loading this
* pool. If so, there's no point in checking the rest of the log
* as its content should have already been synced to the pool.
*/
if (!BP_IS_HOLE(bp)) {
if (!valid)
return (0);
}
/*
* Because tx == NULL, zil_claim_log_block() will not actually claim
* any blocks, but just determine whether it is possible to do so.
* In addition to checking the log chain, zil_claim_log_block()
* will invoke zio_claim() with a done func of spa_claim_notify(),
* which will update spa_max_claim_txg. See spa_load() for details.
*/
}
static int
{
return (-1);
return (1);
return (0);
}
void
{
int i;
if (zfs_nocacheflush)
return;
/*
* Even though we're zl_writer, we still need a lock because the
* zl_get_data() callbacks may have dmu_sync() done callbacks
* that will run concurrently.
*/
for (i = 0; i < ndvas; i++) {
}
}
}
static void
{
/*
* We don't need zl_vdev_lock here because we're the zl_writer,
* and all zl_get_data() callbacks are done.
*/
if (avl_numnodes(t) == 0)
return;
}
/*
* Wait for all the flushes to complete. Not all devices actually
* support the DKIOCFLUSHWRITECACHE ioctl, so it's OK if it fails.
*/
}
/*
* Function called when a log block write completes
*/
static void
{
/*
* Ensure the lwb buffer pointer is cleared before releasing
* the txg. If we have had an allocation failure and
* the txg is waiting to sync then we want want zil_sync()
* to remove the lwb so that it's not picked up as the next new
* one in zil_commit_writer(). zil_sync() will only remove
* the lwb if lwb_buf is null.
*/
/*
* Now that we've written this log block, we have a stable pointer
* to the next block in the chain, so it's OK to let the txg in
* which we allocated the next block sync.
*/
}
/*
* Initialize the io for a log block.
*/
static void
{
}
}
}
/*
* Define a limited set of intent log block sizes.
*
* These must be a multiple of 4KB. Note only the amount used (again
* aligned to 4KB) actually gets written. However, we can't always just
* allocate SPA_OLD_MAXBLOCKSIZE as the slog space could be exhausted.
*/
4096, /* non TX_WRITE */
8192+4096, /* data base */
32*1024 + 4096, /* NFS writes */
};
/*
* Use the slog as long as the logbias is 'latency' and the current commit size
* is less than the limit or the total list size is less than 2X the limit.
* Limit checking is disabled by setting zil_slog_limit to UINT64_MAX.
*/
/*
* Start a log block write and advance to the next log block.
* Calls are serialized.
*/
static lwb_t *
{
int i, error;
} else {
}
/*
* Allocate the next block and save its address in this block
* before writing it in order to establish the log chain.
* Note that if the allocation of nlwb synced before we wrote
* the block that points at it (lwb), we'd leak it if we crashed.
* Therefore, we don't do dmu_tx_commit() until zil_lwb_write_done().
* We dirty the dataset to ensure that zil_sync() will be called
* to clean up in the event of allocation failure or I/O failure.
*/
/*
* Log blocks are pre-allocated. Here we select the size of the next
* block, based on size used in the last block.
* - first find the smallest bucket that will fit the block from a
* limited set of block sizes. This is because it's faster to write
* blocks allocated from the same metaslab as they are adjacent or
* close.
* - next find the maximum from the new suggested size and an array of
* previous sizes. This lessens a picket fence effect of wrongly
* guesssing the size if we have a stream of say 2k, 64k, 2k, 64k
* requests.
*
* Note we only write what is used, but we can't just allocate
* the maximum block size because we can exhaust the available
* pool log space.
*/
for (i = 0; zil_blksz > zil_block_buckets[i]; i++)
continue;
zil_blksz = zil_block_buckets[i];
if (zil_blksz == UINT64_MAX)
for (i = 0; i < ZIL_PREV_BLKS; i++)
/* pass the old blkptr in order to spread log blocks across devs */
if (error == 0) {
/*
* Allocate a new log write buffer (lwb).
*/
/* Record the block for later vdev flushing */
}
/* For Slim ZIL only write what is used. */
} else {
}
/*
* clear unused data for security
*/
/*
* If there was an allocation failure then nlwb will be null which
* forces a txg_wait_synced().
*/
return (nlwb);
}
static lwb_t *
{
char *lr_buf;
return (NULL);
/*
* If this record won't fit in the current log block, start a new one.
*/
return (NULL);
return (lwb);
}
}
/*
* If it's a write, fetch the data or get its blkptr as appropriate.
*/
char *dbuf;
int error;
if (dlen) {
} else {
}
return (lwb);
}
if (error != 0) {
return (lwb);
}
}
}
/*
* We're actually making an entry, so update lrc_seq to be the
* log record sequence number. Note that this is generally not
* equal to the itx sequence number because not all transactions
* are synchronous, and sometimes spa_sync() gets there first.
*/
return (lwb);
}
itx_t *
{
return (itx);
}
void
{
}
/*
* Free up the sync and async itxs. The itxs_t has already been detached
* so no locks are needed.
*/
static void
{
avl_tree_t *t;
void *cookie;
}
t = &itxs->i_async_tree;
}
}
avl_destroy(t);
}
static int
{
return (-1);
return (1);
return (0);
}
/*
* Remove all async itx with the given oid.
*/
static void
{
avl_tree_t *t;
otxg = ZILTEST_TXG;
else
continue;
}
/*
* Locate the object node and append its list.
*/
}
}
}
void
{
/*
* Object ids can be re-instantiated in the next txg so
* remove any async transactions to avoid future leaks.
* This can happen if a fsync occurs on the re-instantiated
* object for a WR_INDIRECT or WR_NEED_COPY write, which gets
* the new file data and flushes a write record for the old object.
*/
/*
* Ensure the data of a renamed file is committed before the rename.
*/
txg = ZILTEST_TXG;
else
/*
* The zil_clean callback hasn't got around to cleaning
* this itxg. Save the itxs for release below.
* This should be rare.
*/
zfs_dbgmsg("zil_itx_assign: missed itx cleanup for "
}
sizeof (itx_async_node_t),
}
} else {
}
}
/* Release the old itxs now we've dropped the lock */
}
/*
* If there are any in-memory intent log transactions which have now been
* synced then start up a taskq to free them. We should only do this after we
* have written out the uberblocks (i.e. txg has been comitted) so that
* don't inadvertently clean out in-memory log records that would be required
* by zil_commit().
*/
void
{
return;
}
/*
* Preferably start a task queue to free up the old itxs but
* if taskq_dispatch can't allocate resources to do that then
* free it in-line. This should be rare. Note, using TQ_SLEEP
* created a bad performance problem.
*/
}
/*
* Get the list of itxs to commit into zl_itx_commit_list.
*/
static void
{
otxg = ZILTEST_TXG;
else
/*
* This is inherently racy, since there is nothing to prevent
* the last synced txg from changing. That's okay since we'll
* only commit things in the future.
*/
continue;
}
/*
* If we're adding itx records to the zl_itx_commit_list,
* then the zil better be dirty in this "txg". We can assert
* that here since we're holding the itxg_lock which will
* prevent spa_sync from cleaning it. Once we add the itxs
* to the zl_itx_commit_list we must commit it to disk even
* if it's unnecessary (i.e. the txg was synced).
*/
}
}
/*
* Move the async itxs for a specified object to commit into sync lists.
*/
static void
{
avl_tree_t *t;
otxg = ZILTEST_TXG;
else
/*
* This is inherently racy, since there is nothing to prevent
* the last synced txg from changing.
*/
continue;
}
/*
* If a foid is specified then find that node and append its
* list. Otherwise walk the tree appending all the lists
* to the sync list. We add to the end rather than the
* beginning to ensure the create has happened.
*/
if (foid != 0) {
}
} else {
}
}
}
}
static void
{
int error = 0;
/*
* Return if there's nothing to commit before we dirty the fs by
* calling zil_create().
*/
return;
}
if (zilog->zl_suspend) {
} else {
}
/*
* This is inherently racy and may result in us writing
* out a log block for a txg that was just synced. This is
* ok since we'll end cleaning up that log block the next
* time we call zil_sync().
*/
}
/* write the last block out */
zilog->zl_cur_used = 0;
/*
* Wait if necessary for the log blocks to be on stable storage.
*/
if (zilog->zl_root_zio) {
}
/*
* Remember the highest committed log sequence number for ztest.
* We only update this value when all the log writes succeeded,
* because ztest wants to ASSERT that it got the whole log chain.
*/
}
/*
* Commit zfs transactions to stable storage.
* If foid is 0 push out all transactions, otherwise push only those
* for that object or might reference that object.
*
* itxs are committed in batches. In a heavily stressed zil there will be
* a commit writer thread who is writing out a bunch of itxs to the log
* for a set of committing threads (cthreads) in the same batch as the writer.
* Those cthreads are all waiting on the same cv for that batch.
*
* There will also be a different and growing batch of threads that are
* waiting to commit (qthreads). When the committing batch completes
* a transition occurs such that the cthreads exit and the qthreads become
* cthreads. One of the new cthreads becomes the writer thread for the
* batch. Any new threads arriving become new qthreads.
*
* Only 2 condition variables are needed and there's no transition
* between the two cvs needed. They just flip-flop between qthreads
* and cthreads.
*
* Using this scheme we can efficiently wakeup up only those threads
* that have been committed.
*/
void
{
return;
/* move the async itxs for the foid to the sync queues */
return;
}
}
zilog->zl_next_batch++;
/* wake up one thread to become the next writer */
/* wake up all threads waiting for this batch to be committed */
}
/*
* Called in syncing context to free committed log blocks and update log header.
*/
void
{
/*
* We don't zero out zl_destroy_txg, so make sure we don't try
* to destroy it twice.
*/
return;
if (*replayed_seq != 0) {
*replayed_seq = 0;
}
if (zilog->zl_keep_first) {
/*
* If this block was part of log chain that couldn't
* be claimed because a device was missing during
* zil_claim(), but that device later returns,
* then this block could erroneously appear valid.
* To guard against this, assign a new GUID to the new
* log chain so it doesn't matter what blk points to.
*/
}
}
break;
/*
* If we don't have anything left in the lwb list then
* we've had an allocation failure and we need to zero
* out the zil_header blkptr so that we don't end
* up freeing the same block twice.
*/
}
}
void
zil_init(void)
{
}
void
zil_fini(void)
{
}
void
{
}
void
{
}
zilog_t *
{
for (int i = 0; i < TXG_SIZE; i++) {
}
return (zilog);
}
void
{
for (int i = 0; i < TXG_SIZE; i++) {
/*
* It's possible for an itx to be generated that doesn't dirty
* a txg (e.g. ztest TX_TRUNCATE). So there's no zil_clean()
* callback to remove the entry. We remove those here.
*
* Also free up the ziltest itxs.
*/
}
}
/*
* Open an intent log.
*/
zilog_t *
{
return (zilog);
}
/*
* Close an intent log.
*/
void
{
/*
* The lwb_max_txg for the stubby lwb will reflect the last activity
* for the zil. After a txg_wait_synced() on the txg we know all the
* callbacks have occurred that may clean the zil. Only then can we
* destroy the zl_clean_taskq.
*/
if (txg)
if (zilog_is_dirty(zilog))
/*
* We should have only one LWB left on the list; remove it now.
*/
}
}
/*
* Suspend an intent log. While in suspended mode, we still honor
* synchronous semantics, but we rely on txg_wait_synced() to do it.
* On old version pools, we suspend the log briefly when taking a
* snapshot so that it will have an empty intent log.
*
* Long holds are not really intended to be used the way we do here --
* held for such a short time. A concurrent caller of dsl_dataset_long_held()
* could fail. Therefore we take pains to only put a long hold if it is
* actually necessary. Fortunately, it will only be necessary if the
* objset is currently mounted (or the ZVOL equivalent). In that case it
* will already have a long hold, so we are not really making things any worse.
*
* Ideally, we would locate the existing long-holder (i.e. the zfsvfs_t or
* zvol_state_t), and use their mechanism to prevent their hold from being
* dropped (e.g. VFS_HOLD()). However, that would be even more pain for
* very little gain.
*
* if cookiep == NULL, this does both the suspend & resume.
* Otherwise, it returns with the dataset "long held", and the cookie
* should be passed into zil_resume().
*/
int
{
int error;
if (error != 0)
return (error);
}
/*
* Don't put a long hold in the cases where we can avoid it. This
* is when there is no cookie so we are doing a suspend & resume
* (i.e. called from zil_vdev_offline()), and there's nothing to do
* for the suspend because it's already suspended, or there's no ZIL.
*/
return (0);
}
zilog->zl_suspend++;
/*
* Someone else is already suspending it.
* Just wait for them to finish.
*/
while (zilog->zl_suspending)
zil_resume(os);
else
return (0);
}
/*
* If there is no pointer to an on-disk block, this ZIL must not
* be active (e.g. filesystem not mounted), so there's nothing
* to clean up.
*/
return (0);
}
zil_commit(zilog, 0);
zil_resume(os);
else
return (0);
}
void
{
zilog->zl_suspend--;
}
typedef struct zil_replay_arg {
void *zr_arg;
char *zr_lr;
static int
{
return (error);
}
static int
{
int error = 0;
return (0);
return (0);
/* Strip case-insensitive bit, still present in log record */
/*
* If this record type can be logged out of order, the object
* (lr_foid) may no longer exist. That's legitimate, not an error.
*/
return (0);
}
/*
* Make a copy of the data so we can revise and extend it.
*/
/*
* If this is a TX_WRITE with a blkptr, suck in the data.
*/
if (error != 0)
}
/*
* The log block containing this lr may have been byteswapped
* so that we can easily examine common fields like lrc_txtype.
* However, the log is a mix of different record types, and only the
* replay vectors know how to byteswap their records. Therefore, if
* the lr was byteswapped, undo it before invoking the replay vector.
*/
if (zr->zr_byteswap)
/*
* We must now do two things atomically: replay this log record,
* and update the log header sequence number to reflect the fact that
* we did so. At the end of each replay function the sequence number
* is updated if we are in replay mode.
*/
if (error != 0) {
/*
* The DMU's dnode layer doesn't see removes until the txg
* commits, so a subsequent claim can spuriously fail with
* EEXIST. So if we receive any error we try syncing out
* any removes then retry the transaction. Note that we
* specify B_FALSE for byteswap now, so we don't do it twice.
*/
if (error != 0)
}
return (0);
}
/* ARGSUSED */
static int
{
zilog->zl_replay_blks++;
return (0);
}
/*
* If this dataset has a non-empty intent log, replay it and destroy it.
*/
void
{
return;
}
/*
* Wait for in-progress removes to sync before starting replay.
*/
zh->zh_claim_txg);
}
{
return (B_TRUE);
return (B_TRUE);
}
return (B_FALSE);
}
/* ARGSUSED */
int
{
int error;
if (error != 0)
return (0);
}