Whamcloud - gitweb
LU-3373 misc: small changes for 3.10 server support
[fs/lustre-release.git] / lustre / osd-ldiskfs / osd_io.c
index 695e883..ace930d 100644 (file)
@@ -27,7 +27,7 @@
  * Copyright (c) 2010, Oracle and/or its affiliates. All rights reserved.
  * Use is subject to license terms.
  *
- * Copyright (c) 2011, 2012, Intel Corporation.
+ * Copyright (c) 2012, 2013, Intel Corporation.
  */
 /*
  * This file is part of Lustre, http://www.lustre.org/
 /* ext_depth() */
 #include <ldiskfs/ldiskfs_extents.h>
 
-#ifndef HAVE_PAGE_CONSTANT
-#define mapping_cap_page_constant_write(mapping) 0
-#define SetPageConstant(page) do {} while (0)
-#define ClearPageConstant(page) do {} while (0)
-#endif
-
-#ifndef HAS_GENERIC_ERROR_REMOVE_PAGE
-int generic_error_remove_page(struct address_space *mapping, struct page *page)
-{
-        if (mapping == NULL)
-                return -EINVAL;
-
-        if (mapping != page->mapping)
-                return -EIO;
-        /*
-         * Only punch for normal data pages for now.
-         * Handling other types like directories would need more auditing.
-         */
-        if (!S_ISREG(mapping->host->i_mode))
-                return -EIO;
-
-        if (page_mapped(page)) {
-                unmap_mapping_range(mapping,
-                                    (loff_t)page->index << PAGE_CACHE_SHIFT,
-                                    PAGE_CACHE_SIZE, 0);
-        }
-        truncate_complete_page(mapping, page);
-        return 0;
-}
-#endif
-
 static int __osd_init_iobuf(struct osd_device *d, struct osd_iobuf *iobuf,
                            int rw, int line, int pages)
 {
@@ -102,18 +71,18 @@ static int __osd_init_iobuf(struct osd_device *d, struct osd_iobuf *iobuf,
                 iobuf->dr_init_at);
        LASSERT(pages <= PTLRPC_MAX_BRW_PAGES);
 
-        cfs_waitq_init(&iobuf->dr_wait);
-        cfs_atomic_set(&iobuf->dr_numreqs, 0);
-        iobuf->dr_npages = 0;
-        iobuf->dr_error = 0;
-        iobuf->dr_dev = d;
-        iobuf->dr_frags = 0;
-        iobuf->dr_elapsed = 0;
-        /* must be counted before, so assert */
-        iobuf->dr_rw = rw;
+       init_waitqueue_head(&iobuf->dr_wait);
+       cfs_atomic_set(&iobuf->dr_numreqs, 0);
+       iobuf->dr_npages = 0;
+       iobuf->dr_error = 0;
+       iobuf->dr_dev = d;
+       iobuf->dr_frags = 0;
+       iobuf->dr_elapsed = 0;
+       /* must be counted before, so assert */
+       iobuf->dr_rw = rw;
        iobuf->dr_init_at = line;
 
-       blocks = pages * (CFS_PAGE_SIZE >> osd_sb(d)->s_blocksize_bits);
+       blocks = pages * (PAGE_CACHE_SIZE >> osd_sb(d)->s_blocksize_bits);
        if (iobuf->dr_bl_buf.lb_len >= blocks * sizeof(iobuf->dr_blocks[0])) {
                LASSERT(iobuf->dr_pg_buf.lb_len >=
                        pages * sizeof(iobuf->dr_pages[0]));
@@ -128,7 +97,7 @@ static int __osd_init_iobuf(struct osd_device *d, struct osd_iobuf *iobuf,
        CDEBUG(D_OTHER, "realloc %u for %u (%u) pages\n",
               (unsigned)(pages * sizeof(iobuf->dr_pages[0])), i, pages);
        pages = i;
-       blocks = pages * (CFS_PAGE_SIZE >> osd_sb(d)->s_blocksize_bits);
+       blocks = pages * (PAGE_CACHE_SIZE >> osd_sb(d)->s_blocksize_bits);
        iobuf->dr_max_pages = 0;
        CDEBUG(D_OTHER, "realloc %u for %u blocks\n",
               (unsigned)(blocks * sizeof(iobuf->dr_blocks[0])), blocks);
@@ -176,13 +145,7 @@ void osd_fini_iobuf(struct osd_device *d, struct osd_iobuf *iobuf)
 #define __REQ_WRITE BIO_RW
 #endif
 
-#ifdef HAVE_BIO_ENDIO_2ARG
-#define DIO_RETURN(a)
 static void dio_complete_routine(struct bio *bio, int error)
-#else
-#define DIO_RETURN(a)   return(a)
-static int dio_complete_routine(struct bio *bio, unsigned int done, int error)
-#endif
 {
         struct osd_iobuf *iobuf = bio->bi_private;
         struct bio_vec *bvl;
@@ -206,7 +169,7 @@ static int dio_complete_routine(struct bio *bio, unsigned int done, int error)
                        bio->bi_rw, bio->bi_vcnt, bio->bi_idx, bio->bi_size,
                        bio->bi_end_io, cfs_atomic_read(&bio->bi_cnt),
                        bio->bi_private);
-                DIO_RETURN(0);
+               return;
         }
 
         /* the check is outside of the cycle for performance reason -bzzz */
@@ -215,18 +178,9 @@ static int dio_complete_routine(struct bio *bio, unsigned int done, int error)
                         if (likely(error == 0))
                                 SetPageUptodate(bvl->bv_page);
                         LASSERT(PageLocked(bvl->bv_page));
-                        ClearPageConstant(bvl->bv_page);
                 }
                 cfs_atomic_dec(&iobuf->dr_dev->od_r_in_flight);
         } else {
-                struct page *p = iobuf->dr_pages[0];
-                if (p->mapping) {
-                        if (mapping_cap_page_constant_write(p->mapping)) {
-                                bio_for_each_segment(bvl, bio, i) {
-                                        ClearPageConstant(bvl->bv_page);
-                                }
-                        }
-                }
                 cfs_atomic_dec(&iobuf->dr_dev->od_w_in_flight);
         }
 
@@ -246,7 +200,7 @@ static int dio_complete_routine(struct bio *bio, unsigned int done, int error)
                iobuf->dr_elapsed_valid = 1;
        }
        if (cfs_atomic_dec_and_test(&iobuf->dr_numreqs))
-               cfs_waitq_signal(&iobuf->dr_wait);
+               wake_up(&iobuf->dr_wait);
 
         /* Completed bios used to be chained off iobuf->dr_bios and freed in
          * filter_clear_dreq().  It was then possible to exhaust the biovec-256
@@ -254,7 +208,6 @@ static int dio_complete_routine(struct bio *bio, unsigned int done, int error)
          * deadlocking the OST.  The bios are now released as soon as complete
          * so the pool cannot be exhausted while IOs are competing. bug 10076 */
         bio_put(bio);
-        DIO_RETURN(0);
 }
 
 static void record_start_io(struct osd_iobuf *iobuf, int size)
@@ -303,7 +256,7 @@ static int can_be_merged(struct bio *bio, sector_t sector)
 static int osd_do_bio(struct osd_device *osd, struct inode *inode,
                       struct osd_iobuf *iobuf)
 {
-        int            blocks_per_page = CFS_PAGE_SIZE >> inode->i_blkbits;
+       int            blocks_per_page = PAGE_CACHE_SIZE >> inode->i_blkbits;
         struct page  **pages = iobuf->dr_pages;
         int            npages = iobuf->dr_npages;
         unsigned long *blocks = iobuf->dr_blocks;
@@ -357,15 +310,6 @@ static int osd_do_bio(struct osd_device *osd, struct inode *inode,
                                 sector_bits))
                                 nblocks++;
 
-                        /* I only set the page to be constant only if it
-                         * is mapped to a contiguous underlying disk block(s).
-                         * It will then make sure the corresponding device
-                         * cache of raid5 will be overwritten by this page.
-                         * - jay */
-                        if (iobuf->dr_rw && (nblocks == blocks_per_page) &&
-                            mapping_cap_page_constant_write(inode->i_mapping))
-                                SetPageConstant(page);
-
                         if (bio != NULL &&
                             can_be_merged(bio, sector) &&
                             bio_add_page(bio, page,
@@ -384,8 +328,7 @@ static int osd_do_bio(struct osd_device *osd, struct inode *inode,
                                        bio->bi_size >> 9, queue_max_sectors(q),
                                        bio_phys_segments(q, bio),
                                        queue_max_phys_segments(q),
-                                       bio_hw_segments(q, bio),
-                                       queue_max_hw_segments(q));
+                                      0, queue_max_hw_segments(q));
 
                                 record_start_io(iobuf, bio->bi_size);
                                 osd_submit_bio(iobuf->dr_rw, bio);
@@ -427,7 +370,7 @@ static int osd_do_bio(struct osd_device *osd, struct inode *inode,
          * parallel and wait for IO completion once transaction is stopped
          * see osd_trans_stop() for more details -bzzz */
         if (iobuf->dr_rw == 0) {
-                cfs_wait_event(iobuf->dr_wait,
+               wait_event(iobuf->dr_wait,
                                cfs_atomic_read(&iobuf->dr_numreqs) == 0);
         }
 
@@ -444,8 +387,8 @@ static int osd_map_remote_to_local(loff_t offset, ssize_t len, int *nrpages,
         *nrpages = 0;
 
         while (len > 0) {
-                int poff = offset & (CFS_PAGE_SIZE - 1);
-                int plen = CFS_PAGE_SIZE - poff;
+               int poff = offset & (PAGE_CACHE_SIZE - 1);
+               int plen = PAGE_CACHE_SIZE - poff;
 
                 if (plen > len)
                         plen = len;
@@ -476,7 +419,7 @@ struct page *osd_get_page(struct dt_object *dt, loff_t offset, int rw)
 
         LASSERT(inode);
 
-        page = find_or_create_page(inode->i_mapping, offset >> CFS_PAGE_SHIFT,
+       page = find_or_create_page(inode->i_mapping, offset >> PAGE_CACHE_SHIFT,
                                    GFP_NOFS | __GFP_HIGHMEM);
         if (unlikely(page == NULL))
                 lprocfs_counter_add(d->od_stats, LPROC_OSD_NO_PAGE, 1);
@@ -567,6 +510,344 @@ static int osd_bufs_put(const struct lu_env *env, struct dt_object *dt,
         RETURN(0);
 }
 
+#ifdef HAVE_EXT_PBLOCK /* Name changed to ext4_ext_pblock for kernel 2.6.35 */
+#define ldiskfs_ext_pblock(ex) ext_pblock((ex))
+#endif
+
+struct bpointers {
+       unsigned long *blocks;
+       unsigned long start;
+       int num;
+       int init_num;
+       int create;
+};
+
+static long ldiskfs_ext_find_goal(struct inode *inode,
+                                 struct ldiskfs_ext_path *path,
+                                 unsigned long block, int *aflags)
+{
+       struct ldiskfs_inode_info *ei = LDISKFS_I(inode);
+       unsigned long bg_start;
+       unsigned long colour;
+       int depth;
+
+       if (path) {
+               struct ldiskfs_extent *ex;
+               depth = path->p_depth;
+
+               /* try to predict block placement */
+               if ((ex = path[depth].p_ext))
+                       return ldiskfs_ext_pblock(ex) +
+                               (block - le32_to_cpu(ex->ee_block));
+
+               /* it looks index is empty
+                * try to find starting from index itself */
+               if (path[depth].p_bh)
+                       return path[depth].p_bh->b_blocknr;
+       }
+
+       /* OK. use inode's group */
+       bg_start = (ei->i_block_group * LDISKFS_BLOCKS_PER_GROUP(inode->i_sb)) +
+               le32_to_cpu(LDISKFS_SB(inode->i_sb)->s_es->s_first_data_block);
+       colour = (current->pid % 16) *
+               (LDISKFS_BLOCKS_PER_GROUP(inode->i_sb) / 16);
+       return bg_start + colour + block;
+}
+
+static unsigned long new_blocks(handle_t *handle, struct inode *inode,
+                               struct ldiskfs_ext_path *path,
+                               unsigned long block, unsigned long *count,
+                               int *err)
+{
+       struct ldiskfs_allocation_request ar;
+       unsigned long pblock;
+       int aflags;
+
+       /* find neighbour allocated blocks */
+       ar.lleft = block;
+       *err = ldiskfs_ext_search_left(inode, path, &ar.lleft, &ar.pleft);
+       if (*err)
+               return 0;
+       ar.lright = block;
+       *err = ldiskfs_ext_search_right(inode, path, &ar.lright, &ar.pright);
+       if (*err)
+               return 0;
+
+       /* allocate new block */
+       ar.goal = ldiskfs_ext_find_goal(inode, path, block, &aflags);
+       ar.inode = inode;
+       ar.logical = block;
+       ar.len = *count;
+       ar.flags = LDISKFS_MB_HINT_DATA;
+       pblock = ldiskfs_mb_new_blocks(handle, &ar, err);
+       *count = ar.len;
+       return pblock;
+}
+
+static int ldiskfs_ext_new_extent_cb(struct inode *inode,
+                                    struct ldiskfs_ext_path *path,
+                                    struct ldiskfs_ext_cache *cex,
+#ifdef HAVE_EXT_PREPARE_CB_EXTENT
+                                    struct ldiskfs_extent *ex,
+#endif
+                                    void *cbdata)
+{
+       struct bpointers *bp = cbdata;
+       struct ldiskfs_extent nex;
+       unsigned long pblock;
+       unsigned long tgen;
+       int err, i;
+       unsigned long count;
+       handle_t *handle;
+
+#ifdef LDISKFS_EXT_CACHE_EXTENT /* until kernel 2.6.37 */
+       if (cex->ec_type == LDISKFS_EXT_CACHE_EXTENT) {
+#else
+       if ((cex->ec_len != 0) && (cex->ec_start != 0)) {
+#endif
+               err = EXT_CONTINUE;
+               goto map;
+       }
+
+       if (bp->create == 0) {
+               i = 0;
+               if (cex->ec_block < bp->start)
+                       i = bp->start - cex->ec_block;
+               if (i >= cex->ec_len)
+                       CERROR("nothing to do?! i = %d, e_num = %u\n",
+                                       i, cex->ec_len);
+               for (; i < cex->ec_len && bp->num; i++) {
+                       *(bp->blocks) = 0;
+                       bp->blocks++;
+                       bp->num--;
+                       bp->start++;
+               }
+
+               return EXT_CONTINUE;
+       }
+
+       tgen = LDISKFS_I(inode)->i_ext_generation;
+       count = ldiskfs_ext_calc_credits_for_insert(inode, path);
+
+       handle = ldiskfs_journal_start(inode, count + LDISKFS_ALLOC_NEEDED + 1);
+       if (IS_ERR(handle)) {
+               return PTR_ERR(handle);
+       }
+
+       if (tgen != LDISKFS_I(inode)->i_ext_generation) {
+               /* the tree has changed. so path can be invalid at moment */
+               ldiskfs_journal_stop(handle);
+               return EXT_REPEAT;
+       }
+
+       /* In 2.6.32 kernel, ldiskfs_ext_walk_space()'s callback func is not
+        * protected by i_data_sem as whole. so we patch it to store
+        * generation to path and now verify the tree hasn't changed */
+       down_write((&LDISKFS_I(inode)->i_data_sem));
+
+       /* validate extent, make sure the extent tree does not changed */
+       if (LDISKFS_I(inode)->i_ext_generation != path[0].p_generation) {
+               /* cex is invalid, try again */
+               up_write(&LDISKFS_I(inode)->i_data_sem);
+               ldiskfs_journal_stop(handle);
+               return EXT_REPEAT;
+       }
+
+       count = cex->ec_len;
+       pblock = new_blocks(handle, inode, path, cex->ec_block, &count, &err);
+       if (!pblock)
+               goto out;
+       BUG_ON(count > cex->ec_len);
+
+       /* insert new extent */
+       nex.ee_block = cpu_to_le32(cex->ec_block);
+       ldiskfs_ext_store_pblock(&nex, pblock);
+       nex.ee_len = cpu_to_le16(count);
+       err = ldiskfs_ext_insert_extent(handle, inode, path, &nex, 0);
+       if (err) {
+               /* free data blocks we just allocated */
+               /* not a good idea to call discard here directly,
+                * but otherwise we'd need to call it every free() */
+               ldiskfs_discard_preallocations(inode);
+#ifdef HAVE_EXT_FREE_BLOCK_WITH_BUFFER_HEAD /* Introduced in 2.6.32-rc7 */
+               ldiskfs_free_blocks(handle, inode, NULL, ldiskfs_ext_pblock(&nex),
+                                   cpu_to_le16(nex.ee_len), 0);
+#else
+               ldiskfs_free_blocks(handle, inode, ldiskfs_ext_pblock(&nex),
+                                   cpu_to_le16(nex.ee_len), 0);
+#endif
+               goto out;
+       }
+
+       /*
+        * Putting len of the actual extent we just inserted,
+        * we are asking ldiskfs_ext_walk_space() to continue
+        * scaning after that block
+        */
+       cex->ec_len = le16_to_cpu(nex.ee_len);
+       cex->ec_start = ldiskfs_ext_pblock(&nex);
+       BUG_ON(le16_to_cpu(nex.ee_len) == 0);
+       BUG_ON(le32_to_cpu(nex.ee_block) != cex->ec_block);
+
+out:
+       up_write((&LDISKFS_I(inode)->i_data_sem));
+       ldiskfs_journal_stop(handle);
+map:
+       if (err >= 0) {
+               /* map blocks */
+               if (bp->num == 0) {
+                       CERROR("hmm. why do we find this extent?\n");
+                       CERROR("initial space: %lu:%u\n",
+                               bp->start, bp->init_num);
+#ifdef LDISKFS_EXT_CACHE_EXTENT /* until kernel 2.6.37 */
+                       CERROR("current extent: %u/%u/%llu %d\n",
+                               cex->ec_block, cex->ec_len,
+                               (unsigned long long)cex->ec_start,
+                               cex->ec_type);
+#else
+                       CERROR("current extent: %u/%u/%llu\n",
+                               cex->ec_block, cex->ec_len,
+                               (unsigned long long)cex->ec_start);
+#endif
+               }
+               i = 0;
+               if (cex->ec_block < bp->start)
+                       i = bp->start - cex->ec_block;
+               if (i >= cex->ec_len)
+                       CERROR("nothing to do?! i = %d, e_num = %u\n",
+                                       i, cex->ec_len);
+               for (; i < cex->ec_len && bp->num; i++) {
+                       *(bp->blocks) = cex->ec_start + i;
+#ifdef LDISKFS_EXT_CACHE_EXTENT /* until kernel 2.6.37 */
+                       if (cex->ec_type != LDISKFS_EXT_CACHE_EXTENT) {
+#else
+                       if ((cex->ec_len == 0) || (cex->ec_start == 0)) {
+#endif
+                               /* unmap any possible underlying metadata from
+                                * the block device mapping.  bug 6998. */
+                               unmap_underlying_metadata(inode->i_sb->s_bdev,
+                                                         *(bp->blocks));
+                       }
+                       bp->blocks++;
+                       bp->num--;
+                       bp->start++;
+               }
+       }
+       return err;
+}
+
+int osd_ldiskfs_map_nblocks(struct inode *inode, unsigned long block,
+                           unsigned long num, unsigned long *blocks,
+                           int create)
+{
+       struct bpointers bp;
+       int err;
+
+       CDEBUG(D_OTHER, "blocks %lu-%lu requested for inode %u\n",
+              block, block + num - 1, (unsigned) inode->i_ino);
+
+       bp.blocks = blocks;
+       bp.start = block;
+       bp.init_num = bp.num = num;
+       bp.create = create;
+
+       err = ldiskfs_ext_walk_space(inode, block, num,
+                                        ldiskfs_ext_new_extent_cb, &bp);
+       ldiskfs_ext_invalidate_cache(inode);
+
+       return err;
+}
+
+int osd_ldiskfs_map_ext_inode_pages(struct inode *inode, struct page **page,
+                                   int pages, unsigned long *blocks,
+                                   int create)
+{
+       int blocks_per_page = PAGE_CACHE_SIZE >> inode->i_blkbits;
+       int rc = 0, i = 0;
+       struct page *fp = NULL;
+       int clen = 0;
+
+       CDEBUG(D_OTHER, "inode %lu: map %d pages from %lu\n",
+               inode->i_ino, pages, (*page)->index);
+
+       /* pages are sorted already. so, we just have to find
+        * contig. space and process them properly */
+       while (i < pages) {
+               if (fp == NULL) {
+                       /* start new extent */
+                       fp = *page++;
+                       clen = 1;
+                       i++;
+                       continue;
+               } else if (fp->index + clen == (*page)->index) {
+                       /* continue the extent */
+                       page++;
+                       clen++;
+                       i++;
+                       continue;
+               }
+
+               /* process found extent */
+               rc = osd_ldiskfs_map_nblocks(inode, fp->index * blocks_per_page,
+                                            clen * blocks_per_page, blocks,
+                                            create);
+               if (rc)
+                       GOTO(cleanup, rc);
+
+               /* look for next extent */
+               fp = NULL;
+               blocks += blocks_per_page * clen;
+       }
+
+       if (fp)
+               rc = osd_ldiskfs_map_nblocks(inode, fp->index * blocks_per_page,
+                                            clen * blocks_per_page, blocks,
+                                            create);
+cleanup:
+       return rc;
+}
+
+int osd_ldiskfs_map_bm_inode_pages(struct inode *inode, struct page **page,
+                                  int pages, unsigned long *blocks,
+                                  int create)
+{
+       int blocks_per_page = PAGE_CACHE_SIZE >> inode->i_blkbits;
+       unsigned long *b;
+       int rc = 0, i;
+
+       for (i = 0, b = blocks; i < pages; i++, page++) {
+               rc = ldiskfs_map_inode_page(inode, *page, b, create);
+               if (rc) {
+                       CERROR("ino %lu, blk %lu create %d: rc %d\n",
+                              inode->i_ino, *b, create, rc);
+                       break;
+               }
+
+               b += blocks_per_page;
+       }
+       return rc;
+}
+
+static int osd_ldiskfs_map_inode_pages(struct inode *inode, struct page **page,
+                                      int pages, unsigned long *blocks,
+                                      int create, struct mutex *optional_mutex)
+{
+       int rc;
+
+       if (LDISKFS_I(inode)->i_flags & LDISKFS_EXTENTS_FL) {
+               rc = osd_ldiskfs_map_ext_inode_pages(inode, page, pages,
+                                                    blocks, create);
+               return rc;
+       }
+       if (optional_mutex != NULL)
+               mutex_lock(optional_mutex);
+       rc = osd_ldiskfs_map_bm_inode_pages(inode, page, pages, blocks, create);
+       if (optional_mutex != NULL)
+               mutex_unlock(optional_mutex);
+
+       return rc;
+}
+
 static int osd_write_prep(const struct lu_env *env, struct dt_object *dt,
                           struct niobuf_local *lnb, int npages)
 {
@@ -590,15 +871,15 @@ static int osd_write_prep(const struct lu_env *env, struct dt_object *dt,
                RETURN(rc);
 
        isize = i_size_read(inode);
-        maxidx = ((isize + CFS_PAGE_SIZE - 1) >> CFS_PAGE_SHIFT) - 1;
+       maxidx = ((isize + PAGE_CACHE_SIZE - 1) >> PAGE_CACHE_SHIFT) - 1;
 
         if (osd->od_writethrough_cache)
                 cache = 1;
         if (isize > osd->od_readcache_max_filesize)
                 cache = 0;
 
-        cfs_gettimeofday(&start);
-        for (i = 0; i < npages; i++) {
+       do_gettimeofday(&start);
+       for (i = 0; i < npages; i++) {
 
                 if (cache == 0)
                         generic_error_remove_page(inode->i_mapping,
@@ -611,7 +892,7 @@ static int osd_write_prep(const struct lu_env *env, struct dt_object *dt,
                  */
                 ClearPageUptodate(lnb[i].page);
 
-                if (lnb[i].len == CFS_PAGE_SIZE)
+               if (lnb[i].len == PAGE_CACHE_SIZE)
                         continue;
 
                 if (maxidx >= lnb[i].page->index) {
@@ -626,19 +907,19 @@ static int osd_write_prep(const struct lu_env *env, struct dt_object *dt,
                        off = (lnb[i].lnb_page_offset + lnb[i].len) &
                              ~CFS_PAGE_MASK;
                         if (off)
-                                memset(p + off, 0, CFS_PAGE_SIZE - off);
+                               memset(p + off, 0, PAGE_CACHE_SIZE - off);
                         kunmap(lnb[i].page);
-                }
-        }
-        cfs_gettimeofday(&end);
-        timediff = cfs_timeval_sub(&end, &start, NULL);
-        lprocfs_counter_add(osd->od_stats, LPROC_OSD_GET_PAGE, timediff);
+               }
+       }
+       do_gettimeofday(&end);
+       timediff = cfs_timeval_sub(&end, &start, NULL);
+       lprocfs_counter_add(osd->od_stats, LPROC_OSD_GET_PAGE, timediff);
 
         if (iobuf->dr_npages) {
-               rc = osd->od_fsops->fs_map_inode_pages(inode, iobuf->dr_pages,
-                                                      iobuf->dr_npages,
-                                                      iobuf->dr_blocks,
-                                                      0, NULL);
+               rc = osd_ldiskfs_map_inode_pages(inode, iobuf->dr_pages,
+                                                iobuf->dr_npages,
+                                                iobuf->dr_blocks,
+                                                0, NULL);
                 if (likely(rc == 0)) {
                         rc = osd_do_bio(osd, inode, iobuf);
                         /* do IO stats for preparation reads */
@@ -704,7 +985,7 @@ static int osd_declare_write_commit(const struct lu_env *env,
                        extents++;
 
                if (!osd_is_mapped(inode, lnb[i].lnb_file_offset))
-                       quota_space += CFS_PAGE_SIZE;
+                       quota_space += PAGE_CACHE_SIZE;
 
                /* ignore quota for the whole request if any page is from
                 * client cache or written by root.
@@ -839,10 +1120,10 @@ static int osd_write_commit(const struct lu_env *env, struct dt_object *dt,
         if (OBD_FAIL_CHECK(OBD_FAIL_OST_MAPBLK_ENOSPC)) {
                 rc = -ENOSPC;
         } else if (iobuf->dr_npages > 0) {
-                rc = osd->od_fsops->fs_map_inode_pages(inode, iobuf->dr_pages,
-                                                      iobuf->dr_npages,
-                                                      iobuf->dr_blocks,
-                                                      1, NULL);
+               rc = osd_ldiskfs_map_inode_pages(inode, iobuf->dr_pages,
+                                                iobuf->dr_npages,
+                                                iobuf->dr_blocks,
+                                                1, NULL);
         } else {
                 /* no pages to write, no transno is needed */
                 thandle->th_local = 1;
@@ -895,8 +1176,8 @@ static int osd_read_prep(const struct lu_env *env, struct dt_object *dt,
        if (i_size_read(inode) > osd->od_readcache_max_filesize)
                cache = 0;
 
-        cfs_gettimeofday(&start);
-        for (i = 0; i < npages; i++) {
+       do_gettimeofday(&start);
+       for (i = 0; i < npages; i++) {
 
                if (i_size_read(inode) <= lnb[i].lnb_file_offset)
                         /* If there's no more data, abort early.
@@ -919,18 +1200,18 @@ static int osd_read_prep(const struct lu_env *env, struct dt_object *dt,
                                             LPROC_OSD_CACHE_MISS, 1);
                         osd_iobuf_add_page(iobuf, lnb[i].page);
                 }
-                if (cache == 0)
-                        generic_error_remove_page(inode->i_mapping,lnb[i].page);
-        }
-        cfs_gettimeofday(&end);
-        timediff = cfs_timeval_sub(&end, &start, NULL);
-        lprocfs_counter_add(osd->od_stats, LPROC_OSD_GET_PAGE, timediff);
+               if (cache == 0)
+                       generic_error_remove_page(inode->i_mapping,lnb[i].page);
+       }
+       do_gettimeofday(&end);
+       timediff = cfs_timeval_sub(&end, &start, NULL);
+       lprocfs_counter_add(osd->od_stats, LPROC_OSD_GET_PAGE, timediff);
 
         if (iobuf->dr_npages) {
-               rc = osd->od_fsops->fs_map_inode_pages(inode, iobuf->dr_pages,
-                                                      iobuf->dr_npages,
-                                                      iobuf->dr_blocks,
-                                                      0, NULL);
+               rc = osd_ldiskfs_map_inode_pages(inode, iobuf->dr_pages,
+                                                iobuf->dr_npages,
+                                                iobuf->dr_blocks,
+                                                0, NULL);
                 rc = osd_do_bio(osd, inode, iobuf);
 
                 /* IO stats will be done in osd_bufs_put() */
@@ -1048,17 +1329,7 @@ static ssize_t osd_declare_write(const struct lu_env *env, struct dt_object *dt,
         oh = container_of0(handle, struct osd_thandle, ot_super);
         LASSERT(oh->ot_handle == NULL);
 
-        /* XXX: size == 0 or INT_MAX indicating a catalog header update or
-         *      llog write, see comment in mdd_declare_llog_record().
-         *
-         *      This hack will be removed with llog over OSD landing
-         */
-        if (size == DECLARE_LLOG_REWRITE)
-                credits = 2;
-        else if (size == DECLARE_LLOG_WRITE)
-                credits = 6;
-        else
-                credits = osd_dto_credits_noquota[DTO_WRITE_BLOCK];
+       credits = osd_dto_credits_noquota[DTO_WRITE_BLOCK];
 
        osd_trans_declare_op(env, oh, OSD_OT_WRITE, credits);
 
@@ -1250,7 +1521,8 @@ static int osd_punch(const struct lu_env *env, struct dt_object *dt,
         struct inode       *inode = obj->oo_inode;
         handle_t           *h;
         tid_t               tid;
-        int                 rc, rc2 = 0;
+       loff_t             oldsize;
+       int                rc = 0, rc2 = 0;
         ENTRY;
 
         LASSERT(end == OBD_OBJECT_EOF);
@@ -1267,13 +1539,25 @@ static int osd_punch(const struct lu_env *env, struct dt_object *dt,
 
         tid = oh->ot_handle->h_transaction->t_tid;
 
-        rc = vmtruncate(inode, start);
+       oldsize=inode->i_size;
+       i_size_write(inode, start);
+       truncate_pagecache(inode, oldsize, start);
+#ifdef HAVE_INODEOPS_TRUNCATE
+       if (inode->i_op->truncate)
+               inode->i_op->truncate(inode);
+#else
+       if (!(inode->i_state & (I_NEW|I_FREEING)))
+               mutex_lock(&inode->i_mutex);
+       ldiskfs_truncate(inode);
+       if (!(inode->i_state & (I_NEW|I_FREEING)))
+               mutex_unlock(&inode->i_mutex);
+#endif
 
         /*
          * For a partial-page truncate, flush the page to disk immediately to
          * avoid data corruption during direct disk write.  b=17397
          */
-        if (rc == 0 && (start & ~CFS_PAGE_MASK) != 0)
+       if ((start & ~CFS_PAGE_MASK) != 0)
                 rc = filemap_fdatawrite_range(inode->i_mapping, start, start+1);
 
         h = journal_current_handle();
@@ -1311,6 +1595,7 @@ static int osd_fiemap_get(const struct lu_env *env, struct dt_object *dt,
         file->f_dentry = dentry;
         file->f_mapping = inode->i_mapping;
         file->f_op = inode->i_fop;
+       set_file_inode(file, inode);
 
         saved_fs = get_fs();
         set_fs(get_ds());
@@ -1342,8 +1627,8 @@ const struct dt_body_operations osd_body_ops = {
         .dbo_declare_write_commit = osd_declare_write_commit,
         .dbo_write_commit         = osd_write_commit,
         .dbo_read_prep            = osd_read_prep,
-        .do_declare_punch         = osd_declare_punch,
-        .do_punch                 = osd_punch,
+        .dbo_declare_punch         = osd_declare_punch,
+        .dbo_punch                 = osd_punch,
         .dbo_fiemap_get           = osd_fiemap_get,
 };